From 7446fad19dee4862167e3e1729b5fb259d8b323f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Wa=C5=9B?= Date: Sun, 10 Nov 2024 11:05:27 +0100 Subject: [PATCH] Support managing views in the Faker connector --- plugin/trino-faker/pom.xml | 6 + .../io/trino/plugin/faker/ColumnInfo.java | 6 + .../io/trino/plugin/faker/FakerMetadata.java | 169 +++++++++- .../trino/plugin/faker/FakerQueryRunner.java | 4 + .../io/trino/plugin/faker/TestFakerViews.java | 291 ++++++++++++++++++ 5 files changed, 460 insertions(+), 16 deletions(-) create mode 100644 plugin/trino-faker/src/test/java/io/trino/plugin/faker/TestFakerViews.java diff --git a/plugin/trino-faker/pom.xml b/plugin/trino-faker/pom.xml index c0d228bead17..12a7d0d47330 100644 --- a/plugin/trino-faker/pom.xml +++ b/plugin/trino-faker/pom.xml @@ -157,6 +157,12 @@ test + + io.trino + trino-testing-services + test + + io.trino trino-tpch diff --git a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/ColumnInfo.java b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/ColumnInfo.java index e2c66de8d1dc..55f84eb9df03 100644 --- a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/ColumnInfo.java +++ b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/ColumnInfo.java @@ -14,6 +14,7 @@ package io.trino.plugin.faker; import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.type.Type; import java.util.Optional; @@ -35,6 +36,11 @@ public String name() return metadata.getName(); } + public Type type() + { + return metadata.getType(); + } + @Override public String toString() { diff --git a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerMetadata.java b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerMetadata.java index 0e3be1ef747f..9312f33e1f41 100644 --- a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerMetadata.java +++ b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerMetadata.java @@ -15,6 +15,7 @@ package io.trino.plugin.faker; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.errorprone.annotations.concurrent.GuardedBy; import io.airlift.slice.Slice; import io.trino.spi.TrinoException; @@ -28,15 +29,16 @@ import io.trino.spi.connector.ConnectorTableLayout; import io.trino.spi.connector.ConnectorTableMetadata; import io.trino.spi.connector.ConnectorTableVersion; +import io.trino.spi.connector.ConnectorViewDefinition; import io.trino.spi.connector.Constraint; import io.trino.spi.connector.ConstraintApplicationResult; import io.trino.spi.connector.LimitApplicationResult; +import io.trino.spi.connector.RelationColumnsMetadata; import io.trino.spi.connector.RetryMode; import io.trino.spi.connector.SaveMode; -import io.trino.spi.connector.SchemaNotFoundException; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.SchemaTablePrefix; -import io.trino.spi.connector.TableColumnsMetadata; +import io.trino.spi.connector.ViewNotFoundException; import io.trino.spi.function.BoundSignature; import io.trino.spi.function.FunctionDependencyDeclaration; import io.trino.spi.function.FunctionId; @@ -59,14 +61,20 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; +import java.util.function.UnaryOperator; import java.util.stream.Collectors; import java.util.stream.Stream; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static com.google.common.collect.Maps.filterKeys; +import static io.trino.spi.StandardErrorCode.ALREADY_EXISTS; import static io.trino.spi.StandardErrorCode.INVALID_COLUMN_PROPERTY; import static io.trino.spi.StandardErrorCode.INVALID_COLUMN_REFERENCE; +import static io.trino.spi.StandardErrorCode.NOT_FOUND; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.StandardErrorCode.SCHEMA_ALREADY_EXISTS; import static io.trino.spi.StandardErrorCode.SCHEMA_NOT_FOUND; @@ -85,11 +93,12 @@ public class FakerMetadata private final List schemas = new ArrayList<>(); private final double nullProbability; private final long defaultLimit; + private final FakerFunctionProvider functionsProvider; @GuardedBy("this") private final Map tables = new HashMap<>(); - - private final FakerFunctionProvider functionsProvider; + @GuardedBy("this") + private final Map views = new HashMap<>(); @Inject public FakerMetadata(FakerConfig config, FakerFunctionProvider functionProvider) @@ -168,7 +177,7 @@ public synchronized ConnectorTableMetadata getTableMetadata( @Override public synchronized List listTables(ConnectorSession session, Optional schemaName) { - return tables.keySet().stream() + return Stream.concat(tables.keySet().stream(), views.keySet().stream()) .filter(table -> schemaName.map(table.getSchemaName()::contentEquals).orElse(true)) .collect(toImmutableList()); } @@ -197,13 +206,33 @@ public synchronized ColumnMetadata getColumnMetadata( } @Override - public synchronized Iterator streamTableColumns(ConnectorSession session, SchemaTablePrefix prefix) + public synchronized Iterator streamRelationColumns( + ConnectorSession session, + Optional schemaName, + UnaryOperator> relationFilter) { - return tables.entrySet().stream() + Map relationColumns = new HashMap<>(); + + SchemaTablePrefix prefix = schemaName.map(SchemaTablePrefix::new) + .orElseGet(SchemaTablePrefix::new); + tables.entrySet().stream() .filter(entry -> prefix.matches(entry.getKey())) - .map(entry -> TableColumnsMetadata.forTable( - entry.getKey(), - entry.getValue().columns().stream().map(ColumnInfo::metadata).collect(toImmutableList()))) + .forEach(entry -> { + SchemaTableName name = entry.getKey(); + RelationColumnsMetadata columns = RelationColumnsMetadata.forTable( + name, + entry.getValue().columns().stream() + .map(ColumnInfo::metadata) + .collect(toImmutableList())); + relationColumns.put(name, columns); + }); + + for (Map.Entry entry : getViews(session, schemaName).entrySet()) { + relationColumns.put(entry.getKey(), RelationColumnsMetadata.forView(entry.getKey(), entry.getValue().getColumns())); + } + + return relationFilter.apply(relationColumns.keySet()).stream() + .map(relationColumns::get) .iterator(); } @@ -219,12 +248,12 @@ public synchronized void renameTable(ConnectorSession session, ConnectorTableHan { checkSchemaExists(newTableName.getSchemaName()); checkTableNotExists(newTableName); + checkViewNotExists(newTableName); FakerTableHandle handle = (FakerTableHandle) tableHandle; SchemaTableName oldTableName = handle.schemaTableName(); - tables.remove(oldTableName); - tables.put(newTableName, tables.get(oldTableName)); + tables.put(newTableName, tables.remove(oldTableName)); } @Override @@ -293,6 +322,7 @@ public synchronized FakerOutputTableHandle beginCreateTable(ConnectorSession ses SchemaTableName tableName = tableMetadata.getTable(); SchemaInfo schema = getSchema(tableName.getSchemaName()); checkTableNotExists(tableMetadata.getTable()); + checkViewNotExists(tableMetadata.getTable()); double schemaNullProbability = (double) schema.properties().getOrDefault(SchemaInfo.NULL_PROBABILITY_PROPERTY, nullProbability); double tableNullProbability = (double) tableMetadata.getProperties().getOrDefault(TableInfo.NULL_PROBABILITY_PROPERTY, schemaNullProbability); @@ -349,7 +379,7 @@ private boolean isCharacterColumn(ColumnMetadata column) private synchronized void checkSchemaExists(String schemaName) { if (schemas.stream().noneMatch(schema -> schema.name().equals(schemaName))) { - throw new SchemaNotFoundException(schemaName); + throw new TrinoException(SCHEMA_NOT_FOUND, format("Schema '%s' does not exist", schemaName)); } } @@ -360,8 +390,19 @@ private synchronized void checkTableNotExists(SchemaTableName tableName) } } + private synchronized void checkViewNotExists(SchemaTableName viewName) + { + if (views.containsKey(viewName)) { + throw new TrinoException(ALREADY_EXISTS, format("View '%s' already exists", viewName)); + } + } + @Override - public synchronized Optional finishCreateTable(ConnectorSession session, ConnectorOutputTableHandle tableHandle, Collection fragments, Collection computedStatistics) + public synchronized Optional finishCreateTable( + ConnectorSession session, + ConnectorOutputTableHandle tableHandle, + Collection fragments, + Collection computedStatistics) { requireNonNull(tableHandle, "tableHandle is null"); FakerOutputTableHandle fakerOutputHandle = (FakerOutputTableHandle) tableHandle; @@ -371,12 +412,108 @@ public synchronized Optional finishCreateTable(Connecto TableInfo info = tables.get(tableName); requireNonNull(info, "info is null"); - // TODO ensure fragments is empty? - tables.put(tableName, new TableInfo(info.columns(), info.properties(), info.comment())); + return Optional.empty(); } + @Override + public synchronized List listViews(ConnectorSession session, Optional schemaName) + { + return views.keySet().stream() + .filter(table -> schemaName.map(table.getSchemaName()::equals).orElse(true)) + .collect(toImmutableList()); + } + + @Override + public synchronized Map getViews(ConnectorSession session, Optional schemaName) + { + SchemaTablePrefix prefix = schemaName.map(SchemaTablePrefix::new).orElseGet(SchemaTablePrefix::new); + return ImmutableMap.copyOf(filterKeys(views, prefix::matches)); + } + + @Override + public synchronized Optional getView(ConnectorSession session, SchemaTableName viewName) + { + return Optional.ofNullable(views.get(viewName)); + } + + @Override + public synchronized void createView( + ConnectorSession session, + SchemaTableName viewName, + ConnectorViewDefinition definition, + Map viewProperties, + boolean replace) + { + checkArgument(viewProperties.isEmpty(), "This connector does not support creating views with properties"); + checkSchemaExists(viewName.getSchemaName()); + checkTableNotExists(viewName); + + if (replace) { + views.put(viewName, definition); + } + else if (views.putIfAbsent(viewName, definition) != null) { + throw new TrinoException(ALREADY_EXISTS, "View '%s' already exists".formatted(viewName)); + } + } + + @Override + public synchronized void setViewComment(ConnectorSession session, SchemaTableName viewName, Optional comment) + { + ConnectorViewDefinition view = getView(session, viewName).orElseThrow(() -> new ViewNotFoundException(viewName)); + views.put(viewName, new ConnectorViewDefinition( + view.getOriginalSql(), + view.getCatalog(), + view.getSchema(), + view.getColumns(), + comment, + view.getOwner(), + view.isRunAsInvoker(), + view.getPath())); + } + + @Override + public synchronized void setViewColumnComment(ConnectorSession session, SchemaTableName viewName, String columnName, Optional comment) + { + ConnectorViewDefinition view = getView(session, viewName).orElseThrow(() -> new ViewNotFoundException(viewName)); + views.put(viewName, new ConnectorViewDefinition( + view.getOriginalSql(), + view.getCatalog(), + view.getSchema(), + view.getColumns().stream() + .map(currentViewColumn -> columnName.equals(currentViewColumn.getName()) ? + new ConnectorViewDefinition.ViewColumn(currentViewColumn.getName(), currentViewColumn.getType(), comment) + : currentViewColumn) + .collect(toImmutableList()), + view.getComment(), + view.getOwner(), + view.isRunAsInvoker(), + view.getPath())); + } + + @Override + public synchronized void renameView(ConnectorSession session, SchemaTableName source, SchemaTableName target) + { + checkSchemaExists(target.getSchemaName()); + if (!views.containsKey(source)) { + throw new TrinoException(NOT_FOUND, "View not found: " + source); + } + checkTableNotExists(target); + + if (views.putIfAbsent(target, views.remove(source)) != null) { + throw new TrinoException(ALREADY_EXISTS, "View '%s' already exists".formatted(target)); + } + } + + @Override + public synchronized void dropView(ConnectorSession session, SchemaTableName viewName) + { + if (views.remove(viewName) == null) { + throw new ViewNotFoundException(viewName); + } + } + @Override public synchronized Map getSchemaProperties(ConnectorSession session, String schemaName) { diff --git a/plugin/trino-faker/src/test/java/io/trino/plugin/faker/FakerQueryRunner.java b/plugin/trino-faker/src/test/java/io/trino/plugin/faker/FakerQueryRunner.java index b64e2303ede4..f591c4ddff7c 100644 --- a/plugin/trino-faker/src/test/java/io/trino/plugin/faker/FakerQueryRunner.java +++ b/plugin/trino-faker/src/test/java/io/trino/plugin/faker/FakerQueryRunner.java @@ -17,6 +17,7 @@ import io.airlift.log.Level; import io.airlift.log.Logger; import io.airlift.log.Logging; +import io.trino.plugin.tpch.TpchPlugin; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; @@ -67,6 +68,9 @@ public DistributedQueryRunner build() queryRunner.installPlugin(new FakerPlugin()); queryRunner.createCatalog(CATALOG, "faker", properties); + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch", ImmutableMap.of()); + return queryRunner; } catch (Exception e) { diff --git a/plugin/trino-faker/src/test/java/io/trino/plugin/faker/TestFakerViews.java b/plugin/trino-faker/src/test/java/io/trino/plugin/faker/TestFakerViews.java new file mode 100644 index 000000000000..809bf8de3302 --- /dev/null +++ b/plugin/trino-faker/src/test/java/io/trino/plugin/faker/TestFakerViews.java @@ -0,0 +1,291 @@ +/* + * 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.faker; + +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.QueryRunner; +import org.intellij.lang.annotations.Language; +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static io.trino.testing.TestingNames.randomNameSuffix; +import static org.assertj.core.api.Assertions.assertThat; + +final class TestFakerViews + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return FakerQueryRunner.builder().build(); + } + + @Test + void testView() + { + @Language("SQL") String query = "SELECT orderkey, orderstatus, (totalprice / 2) half FROM tpch.tiny.orders"; + @Language("SQL") String expectedQuery = "SELECT orderkey, orderstatus, (totalprice / 2) half FROM orders"; + + String catalogName = getSession().getCatalog().orElseThrow(); + String schemaName = getSession().getSchema().orElseThrow(); + String testView = "test_view_" + randomNameSuffix(); + String testViewWithComment = "test_view_with_comment_" + randomNameSuffix(); + assertThat(computeActual("SHOW TABLES").getOnlyColumnAsSet()) // prime the cache, if any + .doesNotContain(testView); + assertUpdate("CREATE VIEW " + testView + " AS SELECT 123 x"); + assertThat(computeActual("SHOW TABLES").getOnlyColumnAsSet()) + .contains(testView); + assertUpdate("CREATE OR REPLACE VIEW " + testView + " AS " + query); + + assertUpdate("CREATE VIEW " + testViewWithComment + " COMMENT 'view comment' AS SELECT 123 x"); + assertUpdate("CREATE OR REPLACE VIEW " + testViewWithComment + " COMMENT 'view comment updated' AS " + query); + + String testViewOriginal = "test_view_original_" + randomNameSuffix(); + String testViewRenamed = "test_view_renamed_" + randomNameSuffix(); + assertUpdate("CREATE VIEW " + testViewOriginal + " AS " + query); + assertUpdate("ALTER VIEW " + testViewOriginal + " RENAME TO " + testViewRenamed); + + // verify comment + assertThat((String) computeScalar("SHOW CREATE VIEW " + testViewWithComment)).contains("COMMENT 'view comment updated'"); + assertThat(query( + """ + SELECT table_name, comment + FROM system.metadata.table_comments + WHERE catalog_name = '%s' AND schema_name = '%s' + """.formatted(catalogName, schemaName))) + .skippingTypesCheck() + .containsAll("VALUES ('" + testView + "', null), ('" + testViewWithComment + "', 'view comment updated')"); + + assertUpdate("COMMENT ON VIEW " + testViewWithComment + " IS 'view comment updated twice'"); + assertThat((String) computeScalar("SHOW CREATE VIEW " + testViewWithComment)).contains("COMMENT 'view comment updated twice'"); + + // reading + assertQuery("SELECT * FROM " + testView, expectedQuery); + assertQuery("SELECT * FROM " + testViewRenamed, expectedQuery); + assertQuery("SELECT * FROM " + testViewWithComment, expectedQuery); + + assertQuery( + """ + SELECT * + FROM %1$s a + JOIN %1$s b on a.orderkey = b.orderkey + """.formatted(testView), + """ + SELECT * + FROM (%1$s) a + JOIN (%1$s) b ON a.orderkey = b.orderkey + """.formatted(expectedQuery)); + + assertQuery( + """ + WITH orders AS ( + SELECT * + FROM tpch.tiny.orders + LIMIT 0 + ) + SELECT * + FROM %s + """.formatted(testView), + expectedQuery); + + assertQuery("SELECT * FROM %s.%s.%s".formatted(catalogName, schemaName, testView), expectedQuery); + + assertUpdate("DROP VIEW " + testViewWithComment); + + // information_schema.views without table_name filter + assertThat(query( + """ + SELECT table_name, regexp_replace(view_definition, '\\s', '') + FROM information_schema.views + WHERE table_schema = '%s' + """.formatted(schemaName))) + .skippingTypesCheck() + .containsAll("VALUES ('%1$s', '%3$s'), ('%2$s', '%3$s')".formatted(testView, testViewRenamed, query.replaceAll("\\s", ""))); + // information_schema.views with table_name filter + assertQuery( + """ + SELECT table_name, regexp_replace(view_definition, '\\s', '') + FROM information_schema.views + WHERE table_schema = '%s' and table_name IN ('%s', '%s') + """.formatted(schemaName, testView, testViewRenamed), + "VALUES ('%1$s', '%3$s'), ('%2$s', '%3$s')".formatted(testView, testViewRenamed, query.replaceAll("\\s", ""))); + + // table listing + assertThat(query("SHOW TABLES")) + .skippingTypesCheck() + .containsAll("VALUES '%s', '%s'".formatted(testView, testViewRenamed)); + // information_schema.tables without table_name filter + assertThat(query( + """ + SELECT table_name, table_type + FROM information_schema.tables + WHERE table_schema = '%s' + """.formatted(schemaName))) + .skippingTypesCheck() + .containsAll("VALUES ('%s', 'VIEW'), ('%s', 'VIEW')".formatted(testView, testViewRenamed)); + // information_schema.tables with table_name filter + assertQuery( + """ + SELECT table_name, table_type + FROM information_schema.tables + WHERE table_schema = '%s' and table_name IN ('%s', '%s') + """.formatted(schemaName, testView, testViewRenamed), + "VALUES ('%s', 'VIEW'), ('%s', 'VIEW')".formatted(testView, testViewRenamed)); + + // system.jdbc.tables without filter + assertThat(query( + """ + SELECT table_schem, table_name, table_type + FROM system.jdbc.tables + """)) + .skippingTypesCheck() + .containsAll("VALUES ('%1$s', '%2$s', 'VIEW'), ('%1$s', '%3$s', 'VIEW')".formatted(schemaName, testView, testViewRenamed)); + + // system.jdbc.tables with table prefix filter + assertQuery( + """ + SELECT table_schem, table_name, table_type + FROM system.jdbc.tables + WHERE table_cat = '%s' AND table_schem = '%s' AND table_name IN ('%s', '%s') + """.formatted(catalogName, schemaName, testView, testViewRenamed), + "VALUES ('%1$s', '%2$s', 'VIEW'), ('%1$s', '%3$s', 'VIEW')".formatted(schemaName, testView, testViewRenamed)); + + // column listing + assertThat(query("SHOW COLUMNS FROM " + testView)) + .result() + .projected("Column") // column types can very between connectors + .skippingTypesCheck() + .matches("VALUES 'orderkey', 'orderstatus', 'half'"); + + assertThat(query("DESCRIBE " + testView)) + .result() + .projected("Column") // column types can very between connectors + .skippingTypesCheck() + .matches("VALUES 'orderkey', 'orderstatus', 'half'"); + + // information_schema.columns without table_name filter + assertThat(query( + """ + SELECT table_name, column_name + FROM information_schema.columns + WHERE table_schema = '%s' + """.formatted(schemaName))) + .skippingTypesCheck() + .containsAll( + """ + SELECT * + FROM (VALUES '%s', '%s') + CROSS JOIN UNNEST(ARRAY['orderkey', 'orderstatus', 'half']) + """.formatted(testView, testViewRenamed)); + + // information_schema.columns with table_name filter + assertThat(query( + """ + SELECT table_name, column_name + FROM information_schema.columns + WHERE table_schema = '%s' and table_name IN ('%s', '%s') + """.formatted(schemaName, testView, testViewRenamed))) + .skippingTypesCheck() + .containsAll( + """ + SELECT * + FROM (VALUES '%s', '%s') + CROSS JOIN UNNEST(ARRAY['orderkey', 'orderstatus', 'half']) + """.formatted(testView, testViewRenamed)); + + // view-specific listings + assertThat(query( + """ + SELECT table_name + FROM information_schema.views + WHERE table_schema = '%s' + """.formatted(schemaName))) + .skippingTypesCheck() + .containsAll("VALUES '%s', '%s'".formatted(testView, testViewRenamed)); + + // system.jdbc.columns without filter + assertThat(query( + """ + SELECT table_schem, table_name, column_name + FROM system.jdbc.columns + """)) + .skippingTypesCheck() + .containsAll( + """ + SELECT * + FROM (VALUES ('%1$s', '%2$s'), ('%1$s', '%3$s')) + CROSS JOIN UNNEST(ARRAY['orderkey', 'orderstatus', 'half']) + """.formatted(schemaName, testView, testViewRenamed)); + + // system.jdbc.columns with schema filter + assertThat(query( + """ + SELECT table_schem, table_name, column_name + FROM system.jdbc.columns + WHERE table_schem LIKE '%%%s%%' + """.formatted(schemaName))) + .skippingTypesCheck() + .containsAll( + """ + SELECT * + FROM (VALUES ('%1$s', '%2$s'), ('%1$s', '%3$s')) + CROSS JOIN UNNEST(ARRAY['orderkey', 'orderstatus', 'half']) + """.formatted(schemaName, testView, testViewRenamed)); + + // system.jdbc.columns with table filter + assertThat(query( + """ + SELECT table_schem, table_name, column_name + FROM system.jdbc.columns + WHERE table_name LIKE '%%%s%%' OR table_name LIKE '%%%s%%' + """.formatted(testView, testViewRenamed))) + .skippingTypesCheck() + .containsAll( + """ + SELECT * + FROM (VALUES ('%1$s', '%2$s'), ('%1$s', '%3$s')) + CROSS JOIN UNNEST(ARRAY['orderkey', 'orderstatus', 'half']) + """.formatted(schemaName, testView, testViewRenamed)); + + assertUpdate("DROP VIEW " + testView); + assertUpdate("DROP VIEW " + testViewRenamed); + assertThat(computeActual("SHOW TABLES").getOnlyColumnAsSet()) + .doesNotContainAnyElementsOf(List.of(testView, testViewRenamed, testViewWithComment)); + } + + @Test + void testViewConflicts() + { + String catalogName = getSession().getCatalog().orElseThrow(); + String schemaName = getSession().getSchema().orElseThrow(); + + String testTable = "test_table_" + randomNameSuffix(); + assertUpdate("CREATE TABLE " + testTable + " (orderkey INT, orderstatus VARCHAR(255), half VARCHAR(255))"); + + assertQueryFails("CREATE VIEW " + testTable + " AS SELECT 123 x", "line 1:1: Table already exists: '%s.%s.%s'".formatted(catalogName, schemaName, testTable)); + + String testView = "test_view_" + randomNameSuffix(); + assertUpdate("CREATE VIEW " + testView + " AS SELECT 123 x"); + + assertQueryFails("CREATE VIEW " + testView + " AS SELECT 123 x", "line 1:1: View already exists: '%s.%s.%s'".formatted(catalogName, schemaName, testView)); + assertQueryFails("CREATE TABLE " + testView + " (orderkey INT, orderstatus VARCHAR(255), half VARCHAR(255))", "View '%s.%s' already exists".formatted(schemaName, testView)); + assertQueryFails("ALTER VIEW " + testView + " RENAME TO " + testTable, "line 1:1: Target view '%s.%s.%s' does not exist, but a table with that name exists.".formatted(catalogName, schemaName, testTable)); + assertQueryFails("ALTER TABLE " + testTable + " RENAME TO " + testView, "line 1:1: Target table '%s.%s.%s' does not exist, but a view with that name exists.".formatted(catalogName, schemaName, testView)); + + assertUpdate("DROP VIEW " + testView); + assertUpdate("DROP TABLE " + testTable); + } +}