Skip to content

Commit

Permalink
Add CREATE CATALOG LIKE implementation
Browse files Browse the repository at this point in the history
  • Loading branch information
ssheikin authored and piotrrzysko committed Jan 2, 2025
1 parent 2152c51 commit 93954a1
Show file tree
Hide file tree
Showing 10 changed files with 493 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -289,6 +289,34 @@ public void createCatalog(CatalogName catalogName, ConnectorName connectorName,
}
}

@Override
public void createCatalogLike(CatalogName oldCatalogName, CatalogName catalogName, boolean notExists, Map<String, Optional<String>> 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)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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<String, Optional<String>> 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)
{
Expand Down
Original file line number Diff line number Diff line change
@@ -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<CreateCatalogLike>
{
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<Void> execute(
CreateCatalogLike statement,
QueryStateMachine stateMachine,
List<Expression> parameters,
WarningCollector warningCollector)
{
Session session = stateMachine.getSession();

CatalogName oldCatalog = new CatalogName(statement.getSource().getValue());
CatalogName catalog = new CatalogName(statement.getTarget().getValue());

Map<String, Optional<String>> properties = getProperties(statement, parameters, session);

accessControl.checkCanCreateCatalog(session.toSecurityContext(), catalog.toString());

catalogManager.createCatalogLike(oldCatalog, catalog, statement.isNotExists(), properties);

return immediateVoidFuture();
}

private Map<String, Optional<String>> getProperties(CreateCatalogLike statement, List<Expression> parameters, Session session)
{
Map<NodeRef<Parameter>, Expression> boundParameters = bindParameters(statement, parameters);
ImmutableMap.Builder<String, Optional<String>> 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();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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<String, Optional<String>> properties)
{
throw new UnsupportedOperationException();
}

@Override
public void renameCatalog(CatalogName catalogName, CatalogName newCatalogName)
{
Expand Down Expand Up @@ -86,6 +92,8 @@ public void dropCatalog(CatalogName catalogName, boolean exists)

void createCatalog(CatalogName catalogName, ConnectorName connectorName, Map<String, String> properties, boolean notExists);

void createCatalogLike(CatalogName oldCatalog, CatalogName catalog, boolean notExists, Map<String, Optional<String>> properties);

void renameCatalog(CatalogName catalogName, CatalogName newCatalogName);

void alterCatalog(CatalogName catalogName, Map<String, Optional<String>> properties);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -204,6 +205,7 @@ private static <T extends Statement> Optional<String> 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());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -1107,6 +1108,13 @@ protected Scope visitDropNotNullConstraint(DropNotNullConstraint node, Optional<
return createAndAssignScope(node, scope);
}

@Override
protected Scope visitCreateCatalogLike(CreateCatalogLike node, Optional<Scope> scope)
{
validateProperties(node.getProperties(), scope);
return createAndAssignScope(node, scope);
}

@Override
protected Scope visitCreateCatalog(CreateCatalog node, Optional<Scope> scope)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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))
Expand Down
Loading

0 comments on commit 93954a1

Please sign in to comment.