Skip to content

Commit

Permalink
Add ALTER TABLE SET PROPERTIES statement
Browse files Browse the repository at this point in the history
Cherry-pick of trinodb/trino@72df2b9

Co-authored-by: ebyhr
  • Loading branch information
pratyakshsharma authored and yingsu00 committed Nov 22, 2024
1 parent 0ed73ea commit 2175ff5
Show file tree
Hide file tree
Showing 41 changed files with 543 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@
import com.facebook.presto.sql.tree.Revoke;
import com.facebook.presto.sql.tree.RevokeRoles;
import com.facebook.presto.sql.tree.Rollback;
import com.facebook.presto.sql.tree.SetProperties;
import com.facebook.presto.sql.tree.SetRole;
import com.facebook.presto.sql.tree.SetSession;
import com.facebook.presto.sql.tree.ShowCatalogs;
Expand Down Expand Up @@ -142,6 +143,7 @@ private StatementUtils() {}
builder.put(DropFunction.class, QueryType.CONTROL);
builder.put(Use.class, QueryType.CONTROL);
builder.put(SetSession.class, QueryType.CONTROL);
builder.put(SetProperties.class, QueryType.DATA_DEFINITION);
builder.put(ResetSession.class, QueryType.CONTROL);
builder.put(StartTransaction.class, QueryType.CONTROL);
builder.put(Commit.class, QueryType.CONTROL);
Expand Down
5 changes: 5 additions & 0 deletions presto-docs/src/main/sphinx/sql/alter-table.rst
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ Synopsis
ALTER TABLE [ IF EXISTS ] name ADD [ CONSTRAINT constraint_name ] { PRIMARY KEY | UNIQUE } ( { column_name [, ...] } ) [ { ENABLED | DISABLED } ] [ [ NOT ] RELY ] [ [ NOT ] ENFORCED } ]
ALTER TABLE [ IF EXISTS ] name DROP CONSTRAINT [ IF EXISTS ] constraint_name
ALTER TABLE [ IF EXISTS ] name ALTER [ COLUMN ] column_name { SET | DROP } NOT NULL
ALTER TABLE [ IF EXISTS ] name SET PROPERTIES (property_name=value, [, ...])
Description
-----------
Expand Down Expand Up @@ -89,6 +90,10 @@ Drop not null constraint from column ``zip`` in the ``users`` table::

ALTER TABLE users ALTER COLUMN zip DROP NOT NULL;

Set table property (``x=y``) to table ``users``::

ALTER TABLE users SET PROPERTIES (x='y');

See Also
--------

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@

import javax.inject.Inject;

import java.util.Map;
import java.util.Optional;
import java.util.Set;

Expand Down Expand Up @@ -101,6 +102,11 @@ public void checkCanCreateTable(ConnectorTransactionHandle transaction, Connecto
{
}

@Override
public void checkCanSetTableProperties(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName, Map<String, Object> properties)
{
}

@Override
public void checkCanDropTable(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@

import javax.inject.Inject;

import java.util.Map;
import java.util.Optional;
import java.util.Set;

Expand Down Expand Up @@ -74,6 +75,7 @@
import static com.facebook.presto.spi.security.AccessDeniedException.denySelectTable;
import static com.facebook.presto.spi.security.AccessDeniedException.denySetCatalogSessionProperty;
import static com.facebook.presto.spi.security.AccessDeniedException.denySetRole;
import static com.facebook.presto.spi.security.AccessDeniedException.denySetTableProperties;
import static com.facebook.presto.spi.security.AccessDeniedException.denyShowRoles;
import static com.facebook.presto.spi.security.AccessDeniedException.denyTruncateTable;
import static com.facebook.presto.spi.security.AccessDeniedException.denyUpdateTableColumns;
Expand Down Expand Up @@ -186,6 +188,24 @@ public void checkCanCreateTable(ConnectorTransactionHandle transaction, Connecto
}
}

@Override
public void checkCanSetTableProperties(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName, Map<String, Object> properties)
{
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(transactionHandle, identity, metastoreContext, tableName)) {
denySetTableProperties(tableName.toString());
}
}

@Override
public void checkCanDropTable(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import com.facebook.presto.spi.security.PrestoPrincipal;
import com.facebook.presto.spi.security.Privilege;

import java.util.Map;
import java.util.Optional;
import java.util.Set;

Expand Down Expand Up @@ -83,6 +84,12 @@ public void checkCanCreateTable(ConnectorTransactionHandle transactionHandle, Co
delegate.checkCanCreateTable(transactionHandle, identity, context, tableName);
}

@Override
public void checkCanSetTableProperties(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName, Map<String, Object> properties)
{
delegate.checkCanSetTableProperties(transactionHandle, identity, context, tableName, properties);
}

@Override
public void checkCanDropTable(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName)
{
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
/*
* 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.metadata.MetadataUtil;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.TableHandle;
import com.facebook.presto.spi.WarningCollector;
import com.facebook.presto.spi.security.AccessControl;
import com.facebook.presto.sql.tree.Expression;
import com.facebook.presto.sql.tree.SetProperties;
import com.facebook.presto.transaction.TransactionManager;
import com.google.common.util.concurrent.ListenableFuture;

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

import static com.facebook.presto.metadata.MetadataUtil.getConnectorIdOrThrow;
import static com.facebook.presto.spi.StandardErrorCode.NOT_FOUND;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.facebook.presto.sql.NodeUtils.mapFromProperties;
import static com.facebook.presto.sql.analyzer.utils.ParameterUtils.parameterExtractor;
import static com.facebook.presto.sql.tree.SetProperties.Type.TABLE;
import static com.google.common.util.concurrent.Futures.immediateFuture;
import static java.lang.String.format;

public class SetPropertiesTask
implements DDLDefinitionTask<SetProperties>
{
@Override
public String getName()
{
return "SET PROPERTIES";
}

@Override
public ListenableFuture<?> execute(SetProperties statement, TransactionManager transactionManager, Metadata metadata, AccessControl accessControl, Session session, List<Expression> parameters, WarningCollector warningCollector)
{
QualifiedObjectName tableName = MetadataUtil.createQualifiedObjectName(session, statement, statement.getTableName());
Map<String, Expression> sqlProperties = mapFromProperties(statement.getProperties());

if (statement.getType() == TABLE) {
Map<String, Object> properties = metadata.getTablePropertyManager().getUserSpecifiedProperties(
getConnectorIdOrThrow(session, metadata, tableName.getCatalogName()),
tableName.getCatalogName(),
sqlProperties,
session,
metadata,
parameterExtractor(statement, parameters)).build();
setTableProperties(statement, tableName, metadata, accessControl, session, properties);
}
else {
throw new PrestoException(NOT_SUPPORTED, format("Unsupported target type: %s", statement.getType()));
}

return immediateFuture(null);
}

private void setTableProperties(SetProperties statement, QualifiedObjectName tableName, Metadata metadata, AccessControl accessControl, Session session, Map<String, Object> properties)
{
if (metadata.getMetadataResolver(session).getMaterializedView(tableName).isPresent()) {
throw new PrestoException(NOT_SUPPORTED, "Cannot set table properties to a materialized view");
}

if (metadata.getMetadataResolver(session).getView(tableName).isPresent()) {
throw new PrestoException(NOT_SUPPORTED, "Cannot set table properties to a view");
}

Optional<TableHandle> tableHandle = metadata.getMetadataResolver(session).getTableHandle(tableName);
if (!tableHandle.isPresent()) {
if (!statement.isTableExists()) {
throw new PrestoException(NOT_FOUND, format("Table does not exist: %s", tableName));
}
return;
}

accessControl.checkCanSetTableProperties(session.getRequiredTransactionId(), session.getIdentity(), session.getAccessControlContext(), tableName, properties);
metadata.setTableProperties(session, tableHandle.get(), properties);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -70,19 +70,15 @@ public final void removeProperties(ConnectorId connectorId)
connectorProperties.remove(connectorId);
}

public final Map<String, Object> getProperties(
public final ImmutableMap.Builder<String, Object> getUserSpecifiedProperties(
ConnectorId connectorId,
String catalog, // only use this for error messages
Map<String, Expression> sqlPropertyValues,
Session session,
Metadata metadata,
Map<NodeRef<Parameter>, Expression> parameters)
{
Map<String, PropertyMetadata<?>> supportedProperties = connectorProperties.get(connectorId);
if (supportedProperties == null) {
throw new PrestoException(NOT_FOUND, "Catalog not found: " + catalog);
}

Map<String, PropertyMetadata<?>> supportedProperties = getSupportedProperties(connectorId, catalog);
ImmutableMap.Builder<String, Object> properties = ImmutableMap.builder();

// Fill in user-specified properties
Expand Down Expand Up @@ -125,6 +121,25 @@ public final Map<String, Object> getProperties(

properties.put(property.getName(), value);
}
return properties;
}

public final Map<String, Object> getProperties(
ConnectorId connectorId,
String catalog, // only use this for error messages
Map<String, Expression> sqlPropertyValues,
Session session,
Metadata metadata,
Map<NodeRef<Parameter>, Expression> parameters)
{
Map<String, PropertyMetadata<?>> supportedProperties = getSupportedProperties(connectorId, catalog);
ImmutableMap.Builder<String, Object> properties = getUserSpecifiedProperties(
connectorId,
catalog,
sqlPropertyValues,
session,
metadata,
parameters);
Map<String, Object> userSpecifiedProperties = properties.build();

// Fill in the remaining properties with non-null defaults
Expand All @@ -144,6 +159,15 @@ public Map<ConnectorId, Map<String, PropertyMetadata<?>>> getAllProperties()
return ImmutableMap.copyOf(connectorProperties);
}

private Map<String, PropertyMetadata<?>> getSupportedProperties(ConnectorId connectorId, String catalog)
{
Map<String, PropertyMetadata<?>> supportedProperties = connectorProperties.get(connectorId);
if (supportedProperties == null) {
throw new PrestoException(NOT_FOUND, "Catalog not found: " + catalog);
}
return supportedProperties;
}

private Object evaluatePropertyValue(Expression expression, Type expectedType, Session session, Metadata metadata, Map<NodeRef<Parameter>, Expression> parameters)
{
Expression rewritten = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(parameters), expression);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -239,6 +239,11 @@ public void renameTable(Session session, TableHandle tableHandle, QualifiedObjec
delegate.renameTable(session, tableHandle, newTableName);
}

public void setTableProperties(Session session, TableHandle tableHandle, Map<String, Object> properties)
{
delegate.setTableProperties(session, tableHandle, properties);
}

@Override
public void renameColumn(Session session, TableHandle tableHandle, ColumnHandle source, String target)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -212,6 +212,11 @@ public interface Metadata
*/
void renameTable(Session session, TableHandle tableHandle, QualifiedObjectName newTableName);

/**
* Set properties to the specified table.
*/
void setTableProperties(Session session, TableHandle tableHandle, Map<String, Object> properties);

/**
* Rename the specified column.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -674,6 +674,14 @@ public void renameTable(Session session, TableHandle tableHandle, QualifiedObjec
metadata.renameTable(session.toConnectorSession(connectorId), tableHandle.getConnectorHandle(), toSchemaTableName(newTableName));
}

@Override
public void setTableProperties(Session session, TableHandle tableHandle, Map<String, Object> properties)
{
ConnectorId connectorId = tableHandle.getConnectorId();
ConnectorMetadata metadata = getMetadataForWrite(session, connectorId);
metadata.setTableProperties(session.toConnectorSession(connectorId), tableHandle.getConnectorHandle(), properties);
}

@Override
public void renameColumn(Session session, TableHandle tableHandle, ColumnHandle source, String target)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import java.util.List;
import java.util.Optional;

import static com.facebook.presto.spi.StandardErrorCode.NOT_FOUND;
import static com.facebook.presto.spi.StandardErrorCode.SYNTAX_ERROR;
import static com.facebook.presto.spi.security.PrincipalType.ROLE;
import static com.facebook.presto.spi.security.PrincipalType.USER;
Expand Down Expand Up @@ -84,6 +85,12 @@ public static SchemaTableName toSchemaTableName(QualifiedObjectName qualifiedObj
return new SchemaTableName(qualifiedObjectName.getSchemaName(), qualifiedObjectName.getObjectName());
}

public static ConnectorId getConnectorIdOrThrow(Session session, Metadata metadata, String catalogName)
{
return metadata.getCatalogHandle(session, catalogName)
.orElseThrow(() -> new PrestoException(NOT_FOUND, "Catalog does not exist: " + catalogName));
}

public static String checkLowerCase(String value, String name)
{
if (value == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -337,6 +337,19 @@ public void checkCanRenameTable(TransactionId transactionId, Identity identity,
}
}

@Override
public void checkCanSetTableProperties(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName tableName, Map<String, Object> properties)
{
requireNonNull(identity, "identity is null");
requireNonNull(tableName, "tableName is null");
authenticationCheck(() -> checkCanAccessCatalog(identity, context, tableName.getCatalogName()));
authorizationCheck(() -> systemAccessControl.get().checkCanSetTableProperties(identity, context, toCatalogSchemaTableName(tableName)));
CatalogAccessControlEntry entry = getConnectorAccessControl(transactionId, tableName.getCatalogName());
if (entry != null) {
authorizationCheck(() -> entry.getAccessControl().checkCanSetTableProperties(entry.getTransactionHandle(transactionId), identity.toConnectorIdentity(tableName.getCatalogName()), context, toSchemaTableName(tableName), properties));
}
}

@Override
public void checkCanShowTablesMetadata(TransactionId transactionId, Identity identity, AccessControlContext context, CatalogSchemaName schema)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,10 @@ public void checkCanCreateTable(Identity identity, AccessControlContext context,
{
}

public void checkCanSetTableProperties(Identity identity, AccessControlContext context, CatalogSchemaTableName table)
{
}

@Override
public void checkCanDropTable(Identity identity, AccessControlContext context, CatalogSchemaTableName table)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@
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.denyRevokeTablePrivilege;
import static com.facebook.presto.spi.security.AccessDeniedException.denySetTableProperties;
import static com.facebook.presto.spi.security.AccessDeniedException.denySetUser;
import static com.facebook.presto.spi.security.AccessDeniedException.denyTruncateTable;
import static com.facebook.presto.spi.security.AccessDeniedException.denyUpdateTableColumns;
Expand Down Expand Up @@ -280,6 +281,13 @@ public void checkCanCreateTable(Identity identity, AccessControlContext context,
}
}

public void checkCanSetTableProperties(Identity identity, AccessControlContext context, CatalogSchemaTableName table)
{
if (!canAccessCatalog(identity, table.getCatalogName(), ALL)) {
denySetTableProperties(table.toString());
}
}

@Override
public void checkCanDropTable(Identity identity, AccessControlContext context, CatalogSchemaTableName table)
{
Expand Down
Loading

0 comments on commit 2175ff5

Please sign in to comment.