From 4b4e4f1c3a161a813757d3c918c371c8f9c48ad1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Osipiuk?= Date: Tue, 13 Jul 2021 16:49:40 +0200 Subject: [PATCH] Add support for Trino views in Iceberg connector --- .../trino/plugin/iceberg/IcebergMetadata.java | 162 +++++++++++++++++- .../iceberg/IcebergMetadataFactory.java | 13 +- .../AbstractTestIcebergConnectorTest.java | 3 + .../TestIcebergConnectorSmokeTest.java | 3 + .../iceberg/TestIcebergMetadataListing.java | 12 +- .../TestIcebergHiveMetadataListing.java | 52 ++++-- 6 files changed, 224 insertions(+), 21 deletions(-) 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 0fa35f4b45ea..fd3a2088a0e9 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 @@ -26,9 +26,13 @@ import io.trino.plugin.base.classloader.ClassLoaderSafeSystemTable; import io.trino.plugin.hive.HdfsEnvironment; import io.trino.plugin.hive.HdfsEnvironment.HdfsContext; +import io.trino.plugin.hive.HiveMetadata; import io.trino.plugin.hive.HiveSchemaProperties; +import io.trino.plugin.hive.HiveViewNotSupportedException; import io.trino.plugin.hive.HiveWrittenPartitions; import io.trino.plugin.hive.TableAlreadyExistsException; +import io.trino.plugin.hive.ViewAlreadyExistsException; +import io.trino.plugin.hive.ViewReaderUtil; import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.Database; @@ -52,6 +56,7 @@ import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTableMetadata; import io.trino.spi.connector.ConnectorTableProperties; +import io.trino.spi.connector.ConnectorViewDefinition; import io.trino.spi.connector.Constraint; import io.trino.spi.connector.ConstraintApplicationResult; import io.trino.spi.connector.DiscretePredicates; @@ -62,6 +67,7 @@ import io.trino.spi.connector.SchemaTablePrefix; import io.trino.spi.connector.SystemTable; import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.connector.ViewNotFoundException; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.NullableValue; import io.trino.spi.predicate.TupleDomain; @@ -116,11 +122,12 @@ import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_METADATA; -import static io.trino.plugin.hive.HiveMetadata.PRESTO_QUERY_ID_NAME; import static io.trino.plugin.hive.HiveMetadata.STORAGE_TABLE; import static io.trino.plugin.hive.HiveMetadata.TABLE_COMMENT; import static io.trino.plugin.hive.HiveType.HIVE_STRING; import static io.trino.plugin.hive.ViewReaderUtil.PRESTO_VIEW_FLAG; +import static io.trino.plugin.hive.ViewReaderUtil.encodeViewData; +import static io.trino.plugin.hive.ViewReaderUtil.isPrestoView; import static io.trino.plugin.hive.metastore.MetastoreUtil.buildInitialPrivilegeSet; import static io.trino.plugin.hive.metastore.StorageFormat.VIEW_STORAGE_FORMAT; import static io.trino.plugin.hive.util.HiveUtil.isHiveSystemSchema; @@ -155,6 +162,7 @@ import static io.trino.spi.StandardErrorCode.INVALID_SCHEMA_PROPERTY; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.StandardErrorCode.SCHEMA_NOT_EMPTY; +import static io.trino.spi.StandardErrorCode.TABLE_NOT_FOUND; import static io.trino.spi.type.BigintType.BIGINT; import static java.util.Collections.singletonList; import static java.util.Objects.requireNonNull; @@ -175,15 +183,23 @@ public class IcebergMetadata private static final String ICEBERG_MATERIALIZED_VIEW_COMMENT = "Presto Materialized View"; public static final String DEPENDS_ON_TABLES = "dependsOnTables"; + // Be compatible with views defined by the Hive connector, which can be useful under certain conditions. + private static final String PRESTO_VIEW_COMMENT = HiveMetadata.PRESTO_VIEW_COMMENT; + private static final String PRESTO_VERSION_NAME = HiveMetadata.PRESTO_VERSION_NAME; + private static final String PRESTO_QUERY_ID_NAME = HiveMetadata.PRESTO_QUERY_ID_NAME; + private static final String PRESTO_VIEW_EXPANDED_TEXT_MARKER = HiveMetadata.PRESTO_VIEW_EXPANDED_TEXT_MARKER; + private final CatalogName catalogName; private final HiveMetastore metastore; private final HdfsEnvironment hdfsEnvironment; private final TypeManager typeManager; private final JsonCodec commitTaskCodec; private final HiveTableOperationsProvider tableOperationsProvider; + private final String trinoVersion; private final Map> snapshotIds = new ConcurrentHashMap<>(); private final Map tableMetadataCache = new ConcurrentHashMap<>(); + private final ViewReaderUtil.PrestoViewReader viewReader = new ViewReaderUtil.PrestoViewReader(); private Transaction transaction; @@ -193,7 +209,8 @@ public IcebergMetadata( HdfsEnvironment hdfsEnvironment, TypeManager typeManager, JsonCodec commitTaskCodec, - HiveTableOperationsProvider tableOperationsProvider) + HiveTableOperationsProvider tableOperationsProvider, + String trinoVersion) { this.catalogName = requireNonNull(catalogName, "catalogName is null"); this.metastore = requireNonNull(metastore, "metastore is null"); @@ -201,6 +218,7 @@ public IcebergMetadata( this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.commitTaskCodec = requireNonNull(commitTaskCodec, "commitTaskCodec is null"); this.tableOperationsProvider = requireNonNull(tableOperationsProvider, "tableOperationsProvider is null"); + this.trinoVersion = requireNonNull(trinoVersion, "trinoVersion is null"); } @Override @@ -770,6 +788,143 @@ public ConnectorTableHandle beginDelete(ConnectorSession session, ConnectorTable throw new TrinoException(NOT_SUPPORTED, "This connector only supports delete where one or more partitions are deleted entirely"); } + @Override + public void createView(ConnectorSession session, SchemaTableName viewName, ConnectorViewDefinition definition, boolean replace) + { + HiveIdentity identity = new HiveIdentity(session); + Map properties = ImmutableMap.builder() + .put(PRESTO_VIEW_FLAG, "true") + .put(PRESTO_VERSION_NAME, trinoVersion) + .put(PRESTO_QUERY_ID_NAME, session.getQueryId()) + .put(TABLE_COMMENT, PRESTO_VIEW_COMMENT) + .build(); + + Table.Builder tableBuilder = Table.builder() + .setDatabaseName(viewName.getSchemaName()) + .setTableName(viewName.getTableName()) + .setOwner(session.getUser()) + .setTableType(org.apache.hadoop.hive.metastore.TableType.VIRTUAL_VIEW.name()) + .setDataColumns(ImmutableList.of(new Column("dummy", HIVE_STRING, Optional.empty()))) + .setPartitionColumns(ImmutableList.of()) + .setParameters(properties) + .setViewOriginalText(Optional.of(encodeViewData(definition))) + .setViewExpandedText(Optional.of(PRESTO_VIEW_EXPANDED_TEXT_MARKER)); + + tableBuilder.getStorageBuilder() + .setStorageFormat(VIEW_STORAGE_FORMAT) + .setLocation(""); + Table table = tableBuilder.build(); + PrincipalPrivileges principalPrivileges = buildInitialPrivilegeSet(session.getUser()); + + Optional existing = metastore.getTable(identity, viewName.getSchemaName(), viewName.getTableName()); + if (existing.isPresent()) { + if (!replace || !isPrestoView(existing.get())) { + throw new ViewAlreadyExistsException(viewName); + } + + metastore.replaceTable(identity, viewName.getSchemaName(), viewName.getTableName(), table, principalPrivileges); + return; + } + + try { + metastore.createTable(identity, table, principalPrivileges); + } + catch (TableAlreadyExistsException e) { + throw new ViewAlreadyExistsException(e.getTableName()); + } + } + + @Override + public void renameView(ConnectorSession session, SchemaTableName source, SchemaTableName target) + { + // Not checking if source view exists as this is already done in RenameViewTask + metastore.renameTable(new HiveIdentity(session), source.getSchemaName(), source.getTableName(), target.getSchemaName(), target.getTableName()); + } + + @Override + public void setViewAuthorization(ConnectorSession session, SchemaTableName viewName, TrinoPrincipal principal) + { + // Not checking if view exists as this is already done in SetViewAuthorizationTask + setTableAuthorization(session, viewName, principal); + } + + @Override + public void dropView(ConnectorSession session, SchemaTableName viewName) + { + if (getView(session, viewName).isEmpty()) { + throw new ViewNotFoundException(viewName); + } + + try { + metastore.dropTable(new HiveIdentity(session), viewName.getSchemaName(), viewName.getTableName(), true); + } + catch (TableNotFoundException e) { + throw new ViewNotFoundException(e.getTableName()); + } + } + + @Override + public List listViews(ConnectorSession session, Optional schemaName) + { + // Filter on PRESTO_VIEW_COMMENT to distinguish from materialized views + return listSchemas(session, schemaName).stream() + .flatMap(schema -> + metastore.getTablesWithParameter(schema, TABLE_COMMENT, PRESTO_VIEW_COMMENT).stream() + .map(table -> new SchemaTableName(schema, table))) + .collect(toImmutableList()); + } + + @Override + public Map getViews(ConnectorSession session, Optional schemaName) + { + ImmutableMap.Builder views = ImmutableMap.builder(); + for (SchemaTableName name : listViews(session, schemaName)) { + try { + getView(session, name).ifPresent(view -> views.put(name, view)); + } + catch (TrinoException e) { + if (e.getErrorCode().equals(TABLE_NOT_FOUND.toErrorCode())) { + // Ignore view that was dropped during query execution (race condition) + } + else { + throw e; + } + } + } + return views.build(); + } + + @Override + public Optional getView(ConnectorSession session, SchemaTableName viewName) + { + if (isHiveSystemSchema(viewName.getSchemaName())) { + return Optional.empty(); + } + return metastore.getTable(new HiveIdentity(session), viewName.getSchemaName(), viewName.getTableName()) + .filter(table -> HiveMetadata.PRESTO_VIEW_COMMENT.equals(table.getParameters().get(TABLE_COMMENT))) // filter out materialized views + .filter(ViewReaderUtil::canDecodeView) + .map(view -> { + if (!isPrestoView(view)) { + throw new HiveViewNotSupportedException(viewName); + } + + ConnectorViewDefinition definition = viewReader + .decodeViewData(view.getViewOriginalText().get(), view, catalogName); + // use owner from table metadata if it exists + if (view.getOwner() != null && !definition.isRunAsInvoker()) { + definition = new ConnectorViewDefinition( + definition.getOriginalSql(), + definition.getCatalog(), + definition.getSchema(), + definition.getColumns(), + definition.getComment(), + Optional.of(view.getOwner()), + false); + } + return definition; + }); + } + @Override public OptionalLong executeDelete(ConnectorSession session, ConnectorTableHandle tableHandle) { @@ -1046,8 +1201,7 @@ private boolean isMaterializedView(Table table) @Override public List listMaterializedViews(ConnectorSession session, Optional schemaName) { - // Iceberg does not support VIEWs - // Filter on ICEBERG_MATERIALIZED_VIEW_COMMENT is used to avoid listing hive views in case of a shared HMS + // Filter on ICEBERG_MATERIALIZED_VIEW_COMMENT is used to avoid listing hive views in case of a shared HMS and to distinguish from standard views return listSchemas(session, schemaName).stream() .flatMap(schema -> metastore.getTablesWithParameter(schema, TABLE_COMMENT, ICEBERG_MATERIALIZED_VIEW_COMMENT).stream() .map(table -> new SchemaTableName(schema, table))) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadataFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadataFactory.java index 11486724dda0..9e95cea028c0 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadataFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadataFactory.java @@ -16,6 +16,7 @@ import io.airlift.json.JsonCodec; import io.trino.plugin.base.CatalogName; import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.NodeVersion; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.spi.type.TypeManager; @@ -31,6 +32,7 @@ public class IcebergMetadataFactory private final TypeManager typeManager; private final JsonCodec commitTaskCodec; private final HiveTableOperationsProvider tableOperationsProvider; + private final String trinoVersion; @Inject public IcebergMetadataFactory( @@ -40,9 +42,10 @@ public IcebergMetadataFactory( HdfsEnvironment hdfsEnvironment, TypeManager typeManager, JsonCodec commitTaskDataJsonCodec, - HiveTableOperationsProvider tableOperationsProvider) + HiveTableOperationsProvider tableOperationsProvider, + NodeVersion nodeVersion) { - this(catalogName, metastore, hdfsEnvironment, typeManager, commitTaskDataJsonCodec, tableOperationsProvider); + this(catalogName, metastore, hdfsEnvironment, typeManager, commitTaskDataJsonCodec, tableOperationsProvider, nodeVersion); } public IcebergMetadataFactory( @@ -51,7 +54,8 @@ public IcebergMetadataFactory( HdfsEnvironment hdfsEnvironment, TypeManager typeManager, JsonCodec commitTaskCodec, - HiveTableOperationsProvider tableOperationsProvider) + HiveTableOperationsProvider tableOperationsProvider, + NodeVersion nodeVersion) { this.catalogName = requireNonNull(catalogName, "catalogName is null"); this.metastore = requireNonNull(metastore, "metastore is null"); @@ -59,10 +63,11 @@ public IcebergMetadataFactory( this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.commitTaskCodec = requireNonNull(commitTaskCodec, "commitTaskCodec is null"); this.tableOperationsProvider = requireNonNull(tableOperationsProvider, "tableOperationsProvider is null"); + this.trinoVersion = requireNonNull(nodeVersion, "nodeVersion is null").toString(); } public IcebergMetadata create() { - return new IcebergMetadata(catalogName, metastore, hdfsEnvironment, typeManager, commitTaskCodec, tableOperationsProvider); + return new IcebergMetadata(catalogName, metastore, hdfsEnvironment, typeManager, commitTaskCodec, tableOperationsProvider, trinoVersion); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/AbstractTestIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/AbstractTestIcebergConnectorTest.java index 915c8c17e76c..2df17a3858aa 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/AbstractTestIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/AbstractTestIcebergConnectorTest.java @@ -121,6 +121,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) case SUPPORTS_TOPN_PUSHDOWN: return false; + case SUPPORTS_CREATE_VIEW: + return true; + case SUPPORTS_CREATE_MATERIALIZED_VIEW: return true; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java index 403eb939032d..b3dea8b69bba 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java @@ -45,6 +45,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) case SUPPORTS_TOPN_PUSHDOWN: return false; + case SUPPORTS_CREATE_VIEW: + return true; + case SUPPORTS_CREATE_MATERIALIZED_VIEW: return true; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java index f8428b30de0c..16bb145c4da9 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java @@ -96,6 +96,7 @@ public void setUp() assertQuerySucceeds("CREATE MATERIALIZED VIEW iceberg.test_schema.iceberg_materialized_view AS " + "SELECT * FROM iceberg.test_schema.iceberg_table1"); storageTable = getStorageTable("iceberg", "test_schema", "iceberg_materialized_view"); + assertQuerySucceeds("CREATE VIEW iceberg.test_schema.iceberg_view AS SELECT * FROM iceberg.test_schema.iceberg_table1"); assertQuerySucceeds("CREATE TABLE hive.test_schema.hive_table (_double DOUBLE)"); assertQuerySucceeds("CREATE VIEW hive.test_schema.hive_view AS SELECT * FROM hive.test_schema.hive_table"); @@ -106,6 +107,7 @@ public void tearDown() { assertQuerySucceeds("DROP TABLE IF EXISTS hive.test_schema.hive_table"); assertQuerySucceeds("DROP VIEW IF EXISTS hive.test_schema.hive_view"); + assertQuerySucceeds("DROP VIEW IF EXISTS iceberg.test_schema.iceberg_view"); assertQuerySucceeds("DROP MATERIALIZED VIEW IF EXISTS iceberg.test_schema.iceberg_materialized_view"); assertQuerySucceeds("DROP TABLE IF EXISTS iceberg.test_schema.iceberg_table2"); assertQuerySucceeds("DROP TABLE IF EXISTS iceberg.test_schema.iceberg_table1"); @@ -121,6 +123,7 @@ public void testTableListing() "iceberg_table2", "iceberg_materialized_view", storageTable.getTableName(), + "iceberg_view", "hive_table", "hive_view"); @@ -130,7 +133,9 @@ public void testTableListing() "'iceberg_table1', " + "'iceberg_table2', " + "'iceberg_materialized_view', " + - "'" + storageTable.getTableName() + "'"); + "'" + storageTable.getTableName() + "', " + + "'iceberg_view', " + + "'hive_view'"); } @Test @@ -146,7 +151,10 @@ public void testTableColumnListing() "('iceberg_materialized_view', '_string'), " + "('iceberg_materialized_view', '_integer'), " + "('" + storageTable.getTableName() + "', '_string'), " + - "('" + storageTable.getTableName() + "', '_integer')"); + "('" + storageTable.getTableName() + "', '_integer'), " + + "('iceberg_view', '_string'), " + + "('iceberg_view', '_integer'), " + + "('hive_view', '_double')"); } @Test diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergHiveMetadataListing.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergHiveMetadataListing.java index 2af719d96a75..b6bd962191cf 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergHiveMetadataListing.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergHiveMetadataListing.java @@ -13,26 +13,44 @@ */ package io.trino.tests.product.iceberg; +import com.google.common.collect.ImmutableList; import io.trino.tempto.AfterTestWithContext; import io.trino.tempto.BeforeTestWithContext; import io.trino.tempto.ProductTest; +import io.trino.tempto.assertions.QueryAssert; import org.testng.annotations.Test; +import java.util.List; + import static com.google.common.collect.Iterators.getOnlyElement; import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; import static io.trino.tests.product.TestGroups.ICEBERG; import static io.trino.tests.product.TestGroups.STORAGE_FORMATS; import static io.trino.tests.product.utils.QueryExecutors.onTrino; +import static java.util.stream.Collectors.toList; public class TestIcebergHiveMetadataListing extends ProductTest { private String storageTable; + private List preexistingTables; + private List preexistingColumns; @BeforeTestWithContext public void setUp() { + cleanUp(); + + preexistingTables = onTrino().executeQuery("SHOW TABLES FROM iceberg.default").rows().stream() + .map(list -> row(list.toArray())) + .collect(toList()); + + preexistingColumns = onTrino().executeQuery("SELECT table_name, column_name FROM iceberg.information_schema.columns " + + "WHERE table_catalog = 'iceberg' AND table_schema = 'default'").rows().stream() + .map(list -> row(list.toArray())) + .collect(toList()); + onTrino().executeQuery("CREATE TABLE iceberg.default.iceberg_table1 (_string VARCHAR, _integer INTEGER)"); onTrino().executeQuery("CREATE MATERIALIZED VIEW iceberg.default.iceberg_materialized_view AS " + "SELECT * FROM iceberg.default.iceberg_table1"); @@ -44,6 +62,7 @@ public void setUp() onTrino().executeQuery("CREATE TABLE hive.default.hive_table (_double DOUBLE)"); onTrino().executeQuery("CREATE VIEW hive.default.hive_view AS SELECT * FROM hive.default.hive_table"); + onTrino().executeQuery("CREATE VIEW iceberg.default.iceberg_view AS SELECT * FROM iceberg.default.iceberg_table1"); } @AfterTestWithContext @@ -51,6 +70,7 @@ public void cleanUp() { onTrino().executeQuery("DROP TABLE IF EXISTS hive.default.hive_table"); onTrino().executeQuery("DROP VIEW IF EXISTS hive.default.hive_view"); + onTrino().executeQuery("DROP VIEW IF EXISTS iceberg.default.iceberg_view"); onTrino().executeQuery("DROP MATERIALIZED VIEW IF EXISTS iceberg.default.iceberg_materialized_view"); onTrino().executeQuery("DROP TABLE IF EXISTS iceberg.default.iceberg_table1"); } @@ -59,10 +79,15 @@ public void cleanUp() public void testTableListing() { assertThat(onTrino().executeQuery("SHOW TABLES FROM iceberg.default")) - .containsOnly( - row("iceberg_table1"), - row("iceberg_materialized_view"), - row(storageTable)); + .containsOnly(ImmutableList.builder() + .addAll(preexistingTables) + .add(row("iceberg_table1")) + .add(row("iceberg_materialized_view")) + .add(row(storageTable)) + .add(row("iceberg_view")) + // Iceberg connector supports Trino views created via Hive connector + .add(row("hive_view")) + .build()); } @Test(groups = {ICEBERG, STORAGE_FORMATS}) @@ -71,12 +96,17 @@ public void testColumnListing() assertThat(onTrino().executeQuery( "SELECT table_name, column_name FROM iceberg.information_schema.columns " + "WHERE table_catalog = 'iceberg' AND table_schema = 'default'")) - .containsOnly( - row("iceberg_table1", "_string"), - row("iceberg_table1", "_integer"), - row("iceberg_materialized_view", "_string"), - row("iceberg_materialized_view", "_integer"), - row(storageTable, "_string"), - row(storageTable, "_integer")); + .containsOnly(ImmutableList.builder() + .addAll(preexistingColumns) + .add(row("iceberg_table1", "_string")) + .add(row("iceberg_table1", "_integer")) + .add(row("iceberg_materialized_view", "_string")) + .add(row("iceberg_materialized_view", "_integer")) + .add(row(storageTable, "_string")) + .add(row(storageTable, "_integer")) + .add(row("iceberg_view", "_string")) + .add(row("iceberg_view", "_integer")) + .add(row("hive_view", "_double")) + .build()); } }