Skip to content

Commit

Permalink
Add support for ALTER VIEW RENAME TO statement
Browse files Browse the repository at this point in the history
  • Loading branch information
dmariamgeorge committed Sep 30, 2024
1 parent 9e2ed29 commit cae013d
Show file tree
Hide file tree
Showing 43 changed files with 515 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
import com.facebook.presto.sql.tree.RenameColumn;
import com.facebook.presto.sql.tree.RenameSchema;
import com.facebook.presto.sql.tree.RenameTable;
import com.facebook.presto.sql.tree.RenameView;
import com.facebook.presto.sql.tree.ResetSession;
import com.facebook.presto.sql.tree.Revoke;
import com.facebook.presto.sql.tree.RevokeRoles;
Expand Down Expand Up @@ -133,6 +134,7 @@ private StatementUtils() {}
builder.put(AddConstraint.class, QueryType.DATA_DEFINITION);
builder.put(AlterColumnNotNull.class, QueryType.DATA_DEFINITION);
builder.put(CreateView.class, QueryType.DATA_DEFINITION);
builder.put(RenameView.class, QueryType.DATA_DEFINITION);
builder.put(TruncateTable.class, QueryType.DATA_DEFINITION);
builder.put(DropView.class, QueryType.DATA_DEFINITION);
builder.put(CreateMaterializedView.class, QueryType.DATA_DEFINITION);
Expand Down
6 changes: 6 additions & 0 deletions presto-docs/src/main/sphinx/connector/iceberg.rst
Original file line number Diff line number Diff line change
Expand Up @@ -1190,6 +1190,12 @@ The table is partitioned by the transformed value of the column::

ALTER TABLE iceberg.web.page_views ADD COLUMN ts timestamp WITH (partitioning = 'hour');

ALTER VIEW

Alter view operations to alter the name of an existing view to a new name is supported in the Iceberg connector::

ALTER TABLE iceberg.web.page_views RENAME TO iceberg.web.page_new_views;

TRUNCATE
^^^^^^^^

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -195,6 +195,11 @@ public void checkCanCreateView(ConnectorTransactionHandle transaction, Connector
{
}

@Override
public void checkCanRenameView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName, SchemaTableName newViewName)
{
}

@Override
public void checkCanDropView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameColumn;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameSchema;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameTable;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameView;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeRoles;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeTablePrivilege;
import static com.facebook.presto.spi.security.AccessDeniedException.denySelectTable;
Expand Down Expand Up @@ -431,6 +432,24 @@ public void checkCanCreateView(ConnectorTransactionHandle transaction, Connector
}
}

@Override
public void checkCanRenameView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName, SchemaTableName newViewName)
{
MetastoreContext metastoreContext = new MetastoreContext(
identity, context.getQueryId().getId(),
context.getClientInfo(),
context.getClientTags(),
context.getSource(),
Optional.empty(),
false,
HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER,
context.getWarningCollector(),
context.getRuntimeStats());
if (!isTableOwner(transaction, identity, metastoreContext, viewName)) {
denyRenameView(viewName.toString(), newViewName.toString());
}
}

@Override
public void checkCanDropView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -166,6 +166,12 @@ public void checkCanCreateView(ConnectorTransactionHandle transactionHandle, Con
delegate.checkCanCreateView(transactionHandle, identity, context, viewName);
}

@Override
public void checkCanRenameView(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName, SchemaTableName newViewName)
{
delegate().checkCanRenameView(transactionHandle, identity, context, viewName, newViewName);
}

@Override
public void checkCanDropView(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -414,6 +414,13 @@ public Map<SchemaTableName, ConnectorViewDefinition> getViews(ConnectorSession s
return views.build();
}

@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(getMetastoreContext(session), source.getSchemaName(), source.getTableName(), target.getSchemaName(), target.getTableName());
}

@Override
public void dropView(ConnectorSession session, SchemaTableName viewName)
{
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* 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 com.facebook.presto.execution;

import com.facebook.presto.Session;
import com.facebook.presto.common.QualifiedObjectName;
import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.spi.WarningCollector;
import com.facebook.presto.spi.analyzer.ViewDefinition;
import com.facebook.presto.spi.security.AccessControl;
import com.facebook.presto.sql.analyzer.SemanticException;
import com.facebook.presto.sql.tree.Expression;
import com.facebook.presto.sql.tree.RenameView;
import com.facebook.presto.transaction.TransactionManager;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.inject.Inject;

import java.util.List;
import java.util.Optional;

import static com.facebook.presto.metadata.MetadataUtil.createQualifiedObjectName;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.MISSING_CATALOG;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.MISSING_TABLE;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.NOT_SUPPORTED;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.TABLE_ALREADY_EXISTS;
import static com.google.common.util.concurrent.Futures.immediateFuture;
import static java.util.Objects.requireNonNull;

public class RenameViewTask
implements DDLDefinitionTask<RenameView>
{
private final Metadata metadata;
private final AccessControl accessControl;

@Inject
public RenameViewTask(Metadata metadata, AccessControl accessControl)
{
this.metadata = requireNonNull(metadata, "metadata is null");
this.accessControl = requireNonNull(accessControl, "accessControl is null");
}
@Override
public String getName()
{
return "RENAME VIEW";
}

public ListenableFuture<?> execute(RenameView statement, TransactionManager transactionManager, Metadata metadata, AccessControl accessControl, Session session, List<Expression> parameters, WarningCollector warningCollector)
{
QualifiedObjectName name = createQualifiedObjectName(session, statement, statement.getSource());
QualifiedObjectName viewName = createQualifiedObjectName(session, statement, statement.getSource());

Optional<ViewDefinition> view = metadata.getMetadataResolver(session).getView(name);
if (!view.isPresent()) {
if (!statement.isExists()) {
throw new SemanticException(MISSING_TABLE, statement, "View '%s' does not exist", name);
}
return immediateFuture(null);
}

QualifiedObjectName target = createQualifiedObjectName(session, statement, statement.getTarget());
if (!metadata.getCatalogHandle(session, target.getCatalogName()).isPresent()) {
throw new SemanticException(MISSING_CATALOG, statement, "Target catalog '%s' does not exist", target.getCatalogName());
}
if (metadata.getMetadataResolver(session).getView(target).isPresent()) {
throw new SemanticException(TABLE_ALREADY_EXISTS, statement, "Target view '%s' already exists", target);
}
if (!viewName.getCatalogName().equals(target.getCatalogName())) {
throw new SemanticException(NOT_SUPPORTED, statement, "View rename across catalogs is not supported");
}

accessControl.checkCanRenameView(session.getRequiredTransactionId(), session.getIdentity(), session.getAccessControlContext(), viewName, target);

metadata.renameView(session, viewName, target);

return immediateFuture(null);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -438,6 +438,12 @@ public void createView(Session session, String catalogName, ConnectorTableMetada
delegate.createView(session, catalogName, viewMetadata, viewData, replace);
}

@Override
public void renameView(Session session, QualifiedObjectName existingViewName, QualifiedObjectName newViewName)
{
delegate.renameView(session, existingViewName, newViewName);
}

@Override
public void dropView(Session session, QualifiedObjectName viewName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -368,6 +368,11 @@ public interface Metadata
*/
void createView(Session session, String catalogName, ConnectorTableMetadata viewMetadata, String viewData, boolean replace);

/**
* Rename the specified view.
*/
void renameView(Session session, QualifiedObjectName existingViewName, QualifiedObjectName newViewName);

/**
* Drops the specified view.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1018,6 +1018,16 @@ public void createView(Session session, String catalogName, ConnectorTableMetada
metadata.createView(session.toConnectorSession(connectorId), viewMetadata, viewData, replace);
}

@Override
public void renameView(Session session, QualifiedObjectName source, QualifiedObjectName target)
{
CatalogMetadata catalogMetadata = getCatalogMetadataForWrite(session, target.getCatalogName());
ConnectorId connectorId = catalogMetadata.getConnectorId();
ConnectorMetadata metadata = catalogMetadata.getMetadata();

metadata.renameView(session.toConnectorSession(connectorId), toSchemaTableName(source), toSchemaTableName(target));
}

@Override
public void dropView(Session session, QualifiedObjectName viewName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -501,6 +501,23 @@ public void checkCanCreateView(TransactionId transactionId, Identity identity, A
}
}

@Override
public void checkCanRenameView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName, QualifiedObjectName newViewName)
{
requireNonNull(context, "context is null");
requireNonNull(viewName, "viewName is null");
requireNonNull(newViewName, "newViewName is null");

authenticationCheck(() -> checkCanAccessCatalog(identity, context, viewName.getCatalogName()));

authorizationCheck(() -> systemAccessControl.get().checkCanRenameView(identity, context, toCatalogSchemaTableName(viewName), toCatalogSchemaTableName(newViewName)));

CatalogAccessControlEntry entry = getConnectorAccessControl(transactionId, viewName.getCatalogName());
if (entry != null) {
authorizationCheck(() -> entry.getAccessControl().checkCanRenameView(entry.getTransactionHandle(transactionId), identity.toConnectorIdentity(viewName.getCatalogName()), context, toSchemaTableName(viewName), toSchemaTableName(newViewName)));
}
}

@Override
public void checkCanDropView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -188,6 +188,11 @@ public void checkCanCreateView(Identity identity, AccessControlContext context,
{
}

@Override
public void checkCanRenameView(Identity identity, AccessControlContext context, CatalogSchemaTableName view, CatalogSchemaTableName newView)
{
}

@Override
public void checkCanDropView(Identity identity, AccessControlContext context, CatalogSchemaTableName view)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameColumn;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameSchema;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameTable;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameView;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeTablePrivilege;
import static com.facebook.presto.spi.security.AccessDeniedException.denySetUser;
import static com.facebook.presto.spi.security.AccessDeniedException.denyTruncateTable;
Expand Down Expand Up @@ -380,6 +381,14 @@ public void checkCanCreateView(Identity identity, AccessControlContext context,
}
}

@Override
public void checkCanRenameView(Identity identity, AccessControlContext context, CatalogSchemaTableName view, CatalogSchemaTableName newView)
{
if (!canAccessCatalog(identity, view.getCatalogName(), ALL)) {
denyRenameView(view.toString(), newView.toString());
}
}

@Override
public void checkCanDropView(Identity identity, AccessControlContext context, CatalogSchemaTableName view)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,7 @@
import com.facebook.presto.sql.tree.RenameColumn;
import com.facebook.presto.sql.tree.RenameSchema;
import com.facebook.presto.sql.tree.RenameTable;
import com.facebook.presto.sql.tree.RenameView;
import com.facebook.presto.sql.tree.ResetSession;
import com.facebook.presto.sql.tree.Return;
import com.facebook.presto.sql.tree.Revoke;
Expand Down Expand Up @@ -1002,6 +1003,12 @@ protected Scope visitAlterColumnNotNull(AlterColumnNotNull node, Optional<Scope>
return createAndAssignScope(node, scope);
}

@Override
protected Scope visitRenameView(RenameView node, Optional<Scope> scope)
{
return createAndAssignScope(node, scope);
}

@Override
protected Scope visitDropView(DropView node, Optional<Scope> scope)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@
import com.facebook.presto.execution.QueryManagerConfig;
import com.facebook.presto.execution.RenameColumnTask;
import com.facebook.presto.execution.RenameTableTask;
import com.facebook.presto.execution.RenameViewTask;
import com.facebook.presto.execution.ResetSessionTask;
import com.facebook.presto.execution.RollbackTask;
import com.facebook.presto.execution.ScheduledSplit;
Expand Down Expand Up @@ -205,6 +206,7 @@
import com.facebook.presto.sql.tree.Prepare;
import com.facebook.presto.sql.tree.RenameColumn;
import com.facebook.presto.sql.tree.RenameTable;
import com.facebook.presto.sql.tree.RenameView;
import com.facebook.presto.sql.tree.ResetSession;
import com.facebook.presto.sql.tree.Rollback;
import com.facebook.presto.sql.tree.SetSession;
Expand Down Expand Up @@ -566,6 +568,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig,
.put(DropMaterializedView.class, new DropMaterializedViewTask())
.put(RenameColumn.class, new RenameColumnTask())
.put(RenameTable.class, new RenameTableTask())
.put(RenameView.class, new RenameViewTask(metadata, accessControl))
.put(ResetSession.class, new ResetSessionTask())
.put(SetSession.class, new SetSessionTask())
.put(Prepare.class, new PrepareTask(sqlParser))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameColumn;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameSchema;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameTable;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameView;
import static com.facebook.presto.spi.security.AccessDeniedException.denySelectColumns;
import static com.facebook.presto.spi.security.AccessDeniedException.denySetCatalogSessionProperty;
import static com.facebook.presto.spi.security.AccessDeniedException.denySetSystemSessionProperty;
Expand All @@ -71,6 +72,7 @@
import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.RENAME_COLUMN;
import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.RENAME_SCHEMA;
import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.RENAME_TABLE;
import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.RENAME_VIEW;
import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.SELECT_COLUMN;
import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.SET_SESSION;
import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.SET_USER;
Expand Down Expand Up @@ -271,6 +273,17 @@ public void checkCanCreateView(TransactionId transactionId, Identity identity, A
}
}

@Override
public void checkCanRenameView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName, QualifiedObjectName newViewName)
{
if (shouldDenyPrivilege(identity.getUser(), viewName.getObjectName(), RENAME_VIEW)) {
denyRenameView(viewName.toString(), newViewName.toString());
}
if (denyPrivileges.isEmpty()) {
super.checkCanRenameView(transactionId, identity, context, viewName, newViewName);
}
}

@Override
public void checkCanDropView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName)
{
Expand Down Expand Up @@ -368,7 +381,7 @@ public enum TestingPrivilegeType
CREATE_TABLE, DROP_TABLE, RENAME_TABLE, INSERT_TABLE, DELETE_TABLE, TRUNCATE_TABLE, UPDATE_TABLE,
ADD_COLUMN, DROP_COLUMN, RENAME_COLUMN, SELECT_COLUMN,
ADD_CONSTRAINT, DROP_CONSTRAINT,
CREATE_VIEW, DROP_VIEW, CREATE_VIEW_WITH_SELECT_COLUMNS,
CREATE_VIEW, RENAME_VIEW, DROP_VIEW, CREATE_VIEW_WITH_SELECT_COLUMNS,
SET_SESSION
}

Expand Down
Loading

0 comments on commit cae013d

Please sign in to comment.