diff --git a/core/trino-main/src/main/java/io/trino/connector/CoordinatorDynamicCatalogManager.java b/core/trino-main/src/main/java/io/trino/connector/CoordinatorDynamicCatalogManager.java index bb602f98d85..aa1530f8d8d 100644 --- a/core/trino-main/src/main/java/io/trino/connector/CoordinatorDynamicCatalogManager.java +++ b/core/trino-main/src/main/java/io/trino/connector/CoordinatorDynamicCatalogManager.java @@ -289,6 +289,34 @@ public void createCatalog(CatalogName catalogName, ConnectorName connectorName, } } + @Override + public void createCatalogLike(CatalogName oldCatalogName, CatalogName catalogName, boolean notExists, Map> properties) + { + requireNonNull(oldCatalogName, "oldCatalogName is null"); + requireNonNull(catalogName, "catalog is null"); + requireNonNull(properties, "properties is null"); + + catalogsUpdateLock.lock(); + try { + checkState(state != State.STOPPED, "ConnectorManager is stopped"); + + if (!activeCatalogs.containsKey(oldCatalogName)) { + throw new TrinoException(NOT_FOUND, format("Catalog '%s' does not exist", oldCatalogName)); + } + if (activeCatalogs.containsKey(catalogName)) { + if (!notExists) { + throw new TrinoException(ALREADY_EXISTS, format("Catalog '%s' already exists", catalogName)); + } + return; + } + CatalogConnector catalog = createCatalogLikeInternal(oldCatalogName, catalogName, properties); + log.info("Added catalog: %s", catalog.getCatalogHandle()); + } + finally { + catalogsUpdateLock.unlock(); + } + } + @Override public void renameCatalog(CatalogName catalogName, CatalogName newCatalogName) { diff --git a/core/trino-main/src/main/java/io/trino/connector/StaticCatalogManager.java b/core/trino-main/src/main/java/io/trino/connector/StaticCatalogManager.java index 3c7ec74f180..48ca342f258 100644 --- a/core/trino-main/src/main/java/io/trino/connector/StaticCatalogManager.java +++ b/core/trino-main/src/main/java/io/trino/connector/StaticCatalogManager.java @@ -237,6 +237,12 @@ public void createCatalog(CatalogName catalogName, ConnectorName connectorName, throw new TrinoException(NOT_SUPPORTED, "CREATE CATALOG is not supported by the static catalog store"); } + @Override + public void createCatalogLike(CatalogName oldCatalog, CatalogName catalog, boolean notExists, Map> properties) + { + throw new TrinoException(NOT_SUPPORTED, "CREATE CATALOG LIKE is not supported by the static catalog store"); + } + @Override public void renameCatalog(CatalogName catalogName, CatalogName newCatalogName) { diff --git a/core/trino-main/src/main/java/io/trino/execution/CreateCatalogLikeTask.java b/core/trino-main/src/main/java/io/trino/execution/CreateCatalogLikeTask.java new file mode 100644 index 00000000000..7059f09a067 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/execution/CreateCatalogLikeTask.java @@ -0,0 +1,112 @@ +/* + * 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.execution; + +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Inject; +import io.trino.Session; +import io.trino.execution.warnings.WarningCollector; +import io.trino.metadata.CatalogManager; +import io.trino.security.AccessControl; +import io.trino.spi.catalog.CatalogName; +import io.trino.sql.PlannerContext; +import io.trino.sql.tree.CreateCatalogLike; +import io.trino.sql.tree.Expression; +import io.trino.sql.tree.NodeRef; +import io.trino.sql.tree.Parameter; +import io.trino.sql.tree.Property; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.util.concurrent.Futures.immediateVoidFuture; +import static io.trino.execution.ParameterExtractor.bindParameters; +import static io.trino.metadata.PropertyUtil.evaluateProperty; +import static io.trino.spi.StandardErrorCode.INVALID_CATALOG_PROPERTY; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.sql.analyzer.ExpressionTreeUtils.extractLocation; +import static java.util.Objects.requireNonNull; + +public class CreateCatalogLikeTask + implements DataDefinitionTask +{ + private final CatalogManager catalogManager; + private final PlannerContext plannerContext; + private final AccessControl accessControl; + + @Inject + public CreateCatalogLikeTask(CatalogManager catalogManager, PlannerContext plannerContext, AccessControl accessControl) + { + this.catalogManager = requireNonNull(catalogManager, "catalogManager is null"); + this.plannerContext = requireNonNull(plannerContext, "plannerContext is null"); + this.accessControl = requireNonNull(accessControl, "accessControl is null"); + } + + @Override + public String getName() + { + return "CREATE CATALOG LIKE"; + } + + @Override + public ListenableFuture execute( + CreateCatalogLike statement, + QueryStateMachine stateMachine, + List parameters, + WarningCollector warningCollector) + { + Session session = stateMachine.getSession(); + + CatalogName oldCatalog = new CatalogName(statement.getSource().getValue()); + CatalogName catalog = new CatalogName(statement.getTarget().getValue()); + + Map> properties = getProperties(statement, parameters, session); + + accessControl.checkCanCreateCatalog(session.toSecurityContext(), catalog.toString()); + + catalogManager.createCatalogLike(oldCatalog, catalog, statement.isNotExists(), properties); + + return immediateVoidFuture(); + } + + private Map> getProperties(CreateCatalogLike statement, List parameters, Session session) + { + Map, Expression> boundParameters = bindParameters(statement, parameters); + ImmutableMap.Builder> propertiesBuilder = ImmutableMap.builder(); + for (Property property : statement.getProperties()) { + String name = property.getName().getValue(); + if (property.isSetToDefault()) { + propertiesBuilder.put(name, Optional.empty()); + } + else { + propertiesBuilder.put( + name, + Optional.of((String) evaluateProperty( + extractLocation(property), + property.getName().getValue(), + VARCHAR, + property.getNonDefaultValue(), + session, + plannerContext, + accessControl, + boundParameters, + INVALID_CATALOG_PROPERTY, + "catalog property"))); + } + } + return propertiesBuilder.buildOrThrow(); + } +} diff --git a/core/trino-main/src/main/java/io/trino/metadata/CatalogManager.java b/core/trino-main/src/main/java/io/trino/metadata/CatalogManager.java index 8dcbb77d8ac..75ad426e3b7 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/CatalogManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/CatalogManager.java @@ -57,6 +57,12 @@ public void createCatalog(CatalogName catalogName, ConnectorName connectorName, throw new UnsupportedOperationException(); } + @Override + public void createCatalogLike(CatalogName oldCatalog, CatalogName catalog, boolean notExists, Map> properties) + { + throw new UnsupportedOperationException(); + } + @Override public void renameCatalog(CatalogName catalogName, CatalogName newCatalogName) { @@ -86,6 +92,8 @@ public void dropCatalog(CatalogName catalogName, boolean exists) void createCatalog(CatalogName catalogName, ConnectorName connectorName, Map properties, boolean notExists); + void createCatalogLike(CatalogName oldCatalog, CatalogName catalog, boolean notExists, Map> properties); + void renameCatalog(CatalogName catalogName, CatalogName newCatalogName); void alterCatalog(CatalogName catalogName, Map> properties); diff --git a/core/trino-main/src/main/java/io/trino/server/QueryExecutionFactoryModule.java b/core/trino-main/src/main/java/io/trino/server/QueryExecutionFactoryModule.java index 389581aae36..f23e375c3b3 100644 --- a/core/trino-main/src/main/java/io/trino/server/QueryExecutionFactoryModule.java +++ b/core/trino-main/src/main/java/io/trino/server/QueryExecutionFactoryModule.java @@ -22,6 +22,7 @@ import io.trino.execution.CallTask; import io.trino.execution.CommentTask; import io.trino.execution.CommitTask; +import io.trino.execution.CreateCatalogLikeTask; import io.trino.execution.CreateCatalogTask; import io.trino.execution.CreateFunctionTask; import io.trino.execution.CreateMaterializedViewTask; @@ -77,6 +78,7 @@ import io.trino.sql.tree.Comment; import io.trino.sql.tree.Commit; import io.trino.sql.tree.CreateCatalog; +import io.trino.sql.tree.CreateCatalogLike; import io.trino.sql.tree.CreateFunction; import io.trino.sql.tree.CreateMaterializedView; import io.trino.sql.tree.CreateRole; @@ -149,6 +151,7 @@ public void configure(Binder binder) bindDataDefinitionTask(binder, executionBinder, Comment.class, CommentTask.class); bindDataDefinitionTask(binder, executionBinder, Commit.class, CommitTask.class); bindDataDefinitionTask(binder, executionBinder, CreateCatalog.class, CreateCatalogTask.class); + bindDataDefinitionTask(binder, executionBinder, CreateCatalogLike.class, CreateCatalogLikeTask.class); bindDataDefinitionTask(binder, executionBinder, CreateFunction.class, CreateFunctionTask.class); bindDataDefinitionTask(binder, executionBinder, CreateRole.class, CreateRoleTask.class); bindDataDefinitionTask(binder, executionBinder, CreateSchema.class, CreateSchemaTask.class); diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/QueryExplainer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/QueryExplainer.java index f7fe7d86e63..b701c6bd9f0 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/QueryExplainer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/QueryExplainer.java @@ -35,6 +35,7 @@ import io.trino.sql.planner.optimizations.PlanOptimizer; import io.trino.sql.planner.planprinter.PlanPrinter; import io.trino.sql.tree.CreateCatalog; +import io.trino.sql.tree.CreateCatalogLike; import io.trino.sql.tree.CreateMaterializedView; import io.trino.sql.tree.CreateSchema; import io.trino.sql.tree.CreateTable; @@ -204,6 +205,7 @@ private static Optional explainDataDefinition(T st return Optional.of(switch (statement) { case CreateCatalog createCatalog -> "CREATE CATALOG " + createCatalog.getCatalogName(); + case CreateCatalogLike createCatalogLike -> "CREATE CATALOG %s LIKE %s".formatted(createCatalogLike.getTarget(), createCatalogLike.getSource()); case DropCatalog dropCatalog -> "DROP CATALOG " + dropCatalog.getCatalogName(); case RenameCatalog renameCatalog -> "ALTER CATALOG %s RENAME TO %s".formatted(((RenameCatalog) statement).getSource(), ((RenameCatalog) statement).getTarget()); case SetCatalogProperties setCatalogProperties -> "ALTER CATALOG %s SET PROPERTIES".formatted(((SetCatalogProperties) statement).getName()); diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java index 24bca078adf..0a31cc0f545 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java @@ -134,6 +134,7 @@ import io.trino.sql.tree.Comment; import io.trino.sql.tree.Commit; import io.trino.sql.tree.CreateCatalog; +import io.trino.sql.tree.CreateCatalogLike; import io.trino.sql.tree.CreateMaterializedView; import io.trino.sql.tree.CreateSchema; import io.trino.sql.tree.CreateTable; @@ -1107,6 +1108,13 @@ protected Scope visitDropNotNullConstraint(DropNotNullConstraint node, Optional< return createAndAssignScope(node, scope); } + @Override + protected Scope visitCreateCatalogLike(CreateCatalogLike node, Optional scope) + { + validateProperties(node.getProperties(), scope); + return createAndAssignScope(node, scope); + } + @Override protected Scope visitCreateCatalog(CreateCatalog node, Optional scope) { diff --git a/core/trino-main/src/main/java/io/trino/util/StatementUtils.java b/core/trino-main/src/main/java/io/trino/util/StatementUtils.java index 0a06c28934a..fa9d933cb82 100644 --- a/core/trino-main/src/main/java/io/trino/util/StatementUtils.java +++ b/core/trino-main/src/main/java/io/trino/util/StatementUtils.java @@ -19,6 +19,7 @@ import io.trino.execution.CallTask; import io.trino.execution.CommentTask; import io.trino.execution.CommitTask; +import io.trino.execution.CreateCatalogLikeTask; import io.trino.execution.CreateCatalogTask; import io.trino.execution.CreateFunctionTask; import io.trino.execution.CreateMaterializedViewTask; @@ -73,6 +74,7 @@ import io.trino.sql.tree.Comment; import io.trino.sql.tree.Commit; import io.trino.sql.tree.CreateCatalog; +import io.trino.sql.tree.CreateCatalogLike; import io.trino.sql.tree.CreateFunction; import io.trino.sql.tree.CreateMaterializedView; import io.trino.sql.tree.CreateRole; @@ -207,6 +209,7 @@ private StatementUtils() {} .add(dataDefinitionStatement(Commit.class, CommitTask.class)) .add(dataDefinitionStatement(CreateMaterializedView.class, CreateMaterializedViewTask.class)) .add(dataDefinitionStatement(CreateCatalog.class, CreateCatalogTask.class)) + .add(dataDefinitionStatement(CreateCatalogLike.class, CreateCatalogLikeTask.class)) .add(dataDefinitionStatement(CreateFunction.class, CreateFunctionTask.class)) .add(dataDefinitionStatement(CreateRole.class, CreateRoleTask.class)) .add(dataDefinitionStatement(CreateSchema.class, CreateSchemaTask.class)) diff --git a/core/trino-main/src/test/java/io/trino/execution/TestCreateCatalogLikeTask.java b/core/trino-main/src/test/java/io/trino/execution/TestCreateCatalogLikeTask.java new file mode 100644 index 00000000000..3e7db4d9832 --- /dev/null +++ b/core/trino-main/src/test/java/io/trino/execution/TestCreateCatalogLikeTask.java @@ -0,0 +1,292 @@ +/* + * 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.execution; + +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Key; +import com.google.inject.TypeLiteral; +import io.trino.client.NodeVersion; +import io.trino.execution.warnings.WarningCollector; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.spi.TrinoException; +import io.trino.spi.resourcegroups.ResourceGroupId; +import io.trino.sql.tree.CreateCatalog; +import io.trino.sql.tree.CreateCatalogLike; +import io.trino.sql.tree.Identifier; +import io.trino.sql.tree.NodeLocation; +import io.trino.sql.tree.Property; +import io.trino.sql.tree.Statement; +import io.trino.sql.tree.StringLiteral; +import io.trino.testing.QueryRunner; +import io.trino.testing.StandaloneQueryRunner; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; + +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static io.airlift.concurrent.MoreFutures.getFutureValue; +import static io.trino.SessionTestUtils.TEST_SESSION; +import static io.trino.execution.querystats.PlanOptimizersStatsCollector.createPlanOptimizersStatsCollector; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static io.trino.testing.TestingSession.testSession; +import static java.util.Collections.emptyList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_METHOD; + +@TestInstance(PER_METHOD) +public class TestCreateCatalogLikeTask +{ + private static final String CONNECTOR_NAME = "tpch"; + + private QueryRunner queryRunner; + + @BeforeEach + public void setUp() + { + queryRunner = new StandaloneQueryRunner(TEST_SESSION); + queryRunner.installPlugin(new TpchPlugin()); + } + + @AfterEach + public void tearDown() + { + try (QueryRunner ignored = queryRunner) { + queryRunner = null; + } + } + + @Test + public void testCreateCatalogLikeNonExisting() + { + String catalogA = "catalog_a_" + randomNameSuffix(); + String catalogB = "catalog_b_" + randomNameSuffix(); + assertThat(catalogExists(catalogA)).isFalse(); + assertThat(catalogExists(catalogB)).isFalse(); + assertThatExceptionOfType(TrinoException.class) + .isThrownBy(() -> executeCreateCatalogLike(catalogA, catalogB, false, ImmutableList.of())) + .withMessage("Catalog '%s' does not exist", catalogA); + } + + @Test + public void testCreateCatalogLikeTheSameName() + { + String catalog = "catalog_" + randomNameSuffix(); + executeCreateCatalog(catalog, ImmutableList.of()); + assertThat(catalogExists(catalog)).isTrue(); + assertThatExceptionOfType(TrinoException.class) + .isThrownBy(() -> executeCreateCatalogLike(catalog, catalog, false, ImmutableList.of())) + .withMessage("Catalog '%s' already exists", catalog); + } + + @Test + public void testCreateCatalogLikeIfNotExists() + { + String catalogA = "catalog_a_" + randomNameSuffix(); + String catalogB = "catalog_b_" + randomNameSuffix(); + executeCreateCatalog(catalogA, ImmutableList.of()); + assertThat(catalogExists(catalogA)).isTrue(); + executeCreateCatalogLike(catalogA, catalogB, false, ImmutableList.of()); + assertThat(catalogExists(catalogB)).isTrue(); + assertThatExceptionOfType(TrinoException.class) + .isThrownBy(() -> executeCreateCatalogLike(catalogA, catalogB, false, ImmutableList.of())) + .withMessage("Catalog '%s' already exists", catalogB); + executeCreateCatalogLike(catalogA, catalogB, true, ImmutableList.of()); + assertThat(catalogExists(catalogB)).isTrue(); + } + + @Test + public void testCreateCatalogLikeWithoutChangingProperties() + { + testCreateCatalogLike( + ImmutableList.of( + new Property(new Identifier("property0"), new StringLiteral("value0")), + new Property(new Identifier("property1"), new StringLiteral("value1"))), + """ + property0 = 'value0', + property1 = 'value1' + """, + ImmutableList.of(), + """ + property0 = 'value0', + property1 = 'value1' + """); + } + + @Test + public void testAddCatalogProperties() + { + testCreateCatalogLike( + ImmutableList.of( + new Property(new Identifier("property1"), new StringLiteral("value1"))), + """ + property1 = 'value1' + """, + ImmutableList.of( + new Property(new Identifier("property0"), new StringLiteral("value0")), + new Property(new Identifier("property2"), new StringLiteral("value2"))), + """ + property0 = 'value0', + property1 = 'value1', + property2 = 'value2' + """); + } + + @Test + public void testOverrideCatalogProperties() + { + testCreateCatalogLike( + ImmutableList.of( + new Property(new Identifier("property0"), new StringLiteral("value0")), + new Property(new Identifier("property1"), new StringLiteral("value1"))), + """ + property0 = 'value0', + property1 = 'value1' + """, + ImmutableList.of( + new Property(new Identifier("property0"), new StringLiteral("value000")), + new Property(new Identifier("property1"), new StringLiteral("value111"))), + """ + property0 = 'value000', + property1 = 'value111' + """); + } + + @Test + public void testRemoveCatalogProperties() + { + testCreateCatalogLike( + ImmutableList.of( + new Property(new Identifier("property0"), new StringLiteral("value0")), + new Property(new Identifier("property1"), new StringLiteral("value1")), + new Property(new Identifier("property2"), new StringLiteral("value2"))), + """ + property0 = 'value0', + property1 = 'value1', + property2 = 'value2' + """, + ImmutableList.of( + new Property(new Identifier("property2")), + new Property(new Identifier("property0"))), + """ + property1 = 'value1' + """); + } + + @Test + public void testSetComplexCatalogProperties() + { + testCreateCatalogLike( + ImmutableList.of( + new Property(new Identifier("property1-to-remove"), new StringLiteral("value-to-remove")), + new Property(new Identifier("property2"), new StringLiteral("value not changed")), + new Property(new Identifier("property3-to-update"), new StringLiteral("value-old"))), + """ + "property1-to-remove" = 'value-to-remove', + property2 = 'value not changed', + "property3-to-update" = 'value-old' + """, + ImmutableList.of( + new Property(new Identifier("property0-added"), new StringLiteral("value-added")), + new Property(new Identifier("property1-to-remove")), + new Property(new Identifier("property3-to-update"), new StringLiteral("value-updated")), + new Property(new Identifier("property4-added"), new StringLiteral("${ENV:foo}")), + new Property(new Identifier("property5-added"), new StringLiteral("${FILE:bar}"))), + """ + "property0-added" = 'value-added', + property2 = 'value not changed', + "property3-to-update" = 'value-updated', + "property4-added" = '${ENV:foo}', + "property5-added" = '${FILE:bar}' + """); + } + + private void testCreateCatalogLike(ImmutableList initialProperties, String showInitialProperties, List updatedProperties, String showExpectedProperties) + { + String createCatalogSql = """ + CREATE CATALOG %s USING %s + WITH ( + %s)"""; + + String oldCatalog = "catalog_" + randomNameSuffix(); + executeCreateCatalog(oldCatalog, initialProperties); + assertThat(catalogExists(oldCatalog)).isTrue(); + assertThat((String) queryRunner.execute("SHOW CREATE CATALOG " + oldCatalog).getOnlyValue()) + .isEqualTo(createCatalogSql.formatted(oldCatalog, CONNECTOR_NAME, showInitialProperties)); + + String catalog = "catalog_" + randomNameSuffix(); + executeCreateCatalogLike(oldCatalog, catalog, false, updatedProperties); + assertThat(catalogExists(catalog)).isTrue(); + assertThat((String) queryRunner.execute("SHOW CREATE CATALOG " + catalog).getOnlyValue()) + .isEqualTo(createCatalogSql.formatted(catalog, CONNECTOR_NAME, showExpectedProperties)); + } + + private void executeCreateCatalog(String catalogA, List catalogProperties) + { + Map, DataDefinitionTask> tasks = queryRunner.getCoordinator().getInstance(Key.get(new TypeLiteral<>() {})); + CreateCatalogTask task = (CreateCatalogTask) tasks.get(CreateCatalog.class); + CreateCatalog statement = new CreateCatalog( + new NodeLocation(1, 1), + new Identifier(catalogA), + false, + new Identifier(CONNECTOR_NAME), + catalogProperties, + Optional.empty(), + Optional.empty()); + ListenableFuture future = task.execute(statement, createNewQuery(), emptyList(), WarningCollector.NOOP); + getFutureValue(future); + } + + private void executeCreateCatalogLike(String oldCatalog, String catalog, boolean notExists, List properties) + { + Map, DataDefinitionTask> tasks = queryRunner.getCoordinator().getInstance(Key.get(new TypeLiteral<>() {})); + CreateCatalogLikeTask task = (CreateCatalogLikeTask) tasks.get(CreateCatalogLike.class); + CreateCatalogLike statement = new CreateCatalogLike(new Identifier(oldCatalog), new Identifier(catalog), notExists, properties); + ListenableFuture future = task.execute(statement, createNewQuery(), emptyList(), WarningCollector.NOOP); + getFutureValue(future); + } + + private boolean catalogExists(String catalogB) + { + return queryRunner.getPlannerContext().getMetadata().catalogExists(createNewQuery().getSession(), catalogB); + } + + private QueryStateMachine createNewQuery() + { + return QueryStateMachine.begin( + Optional.empty(), + "test", + Optional.empty(), + testSession(queryRunner.getDefaultSession()), + URI.create("fake://uri"), + new ResourceGroupId("test"), + false, + queryRunner.getTransactionManager(), + queryRunner.getAccessControl(), + directExecutor(), + queryRunner.getPlannerContext().getMetadata(), + WarningCollector.NOOP, + createPlanOptimizersStatsCollector(), + Optional.empty(), + true, + new NodeVersion("test")); + } +} diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDynamicCatalogsSql.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDynamicCatalogsSql.java index 3a7dbb931c2..b101856ce95 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDynamicCatalogsSql.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDynamicCatalogsSql.java @@ -57,6 +57,37 @@ public void testCreateDropCatalog() assertUpdate("DROP CATALOG " + catalog); } + @Test + public void testCreateCatalogLike() + { + String createCatalogSql = """ + CREATE CATALOG %s USING base_jdbc + WITH ( + "bootstrap.quiet" = 'true', + "connection-url" = '%s' + )"""; + + String oldCatalog = "catalog_" + randomNameSuffix(); + String oldConnectionUrl = createH2ConnectionUrl(); + assertUpdate(createCatalogSql.formatted(oldCatalog, oldConnectionUrl)); + + String catalog = "catalog_" + randomNameSuffix(); + String newConnectionUrl = createH2ConnectionUrl(); + assertUpdate(""" + CREATE CATALOG %s LIKE %s + WITH ( + "bootstrap.quiet" = 'true', + "connection-url" = '%s' + ) + """ + .formatted(catalog, oldCatalog, newConnectionUrl)); + assertThat((String) computeActual("SHOW CATALOGS LIKE '%s'".formatted(catalog)).getOnlyValue()) + .isEqualTo(catalog); + + assertUpdate("DROP CATALOG " + catalog); + assertUpdate("DROP CATALOG " + oldCatalog); + } + @Test public void testRenameCatalog() {