Skip to content

Commit 8a539d8

Browse files
Add DROP TAG sql support
1 parent 166ed0a commit 8a539d8

File tree

36 files changed

+441
-2
lines changed

36 files changed

+441
-2
lines changed

presto-analyzer/src/main/java/com/facebook/presto/sql/analyzer/utils/StatementUtils.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@
4141
import com.facebook.presto.sql.tree.DropRole;
4242
import com.facebook.presto.sql.tree.DropSchema;
4343
import com.facebook.presto.sql.tree.DropTable;
44+
import com.facebook.presto.sql.tree.DropTag;
4445
import com.facebook.presto.sql.tree.DropView;
4546
import com.facebook.presto.sql.tree.Explain;
4647
import com.facebook.presto.sql.tree.Grant;
@@ -131,6 +132,7 @@ private StatementUtils() {}
131132
builder.put(DropColumn.class, QueryType.DATA_DEFINITION);
132133
builder.put(DropTable.class, QueryType.DATA_DEFINITION);
133134
builder.put(DropBranch.class, QueryType.DATA_DEFINITION);
135+
builder.put(DropTag.class, QueryType.DATA_DEFINITION);
134136
builder.put(DropConstraint.class, QueryType.DATA_DEFINITION);
135137
builder.put(AddConstraint.class, QueryType.DATA_DEFINITION);
136138
builder.put(AlterColumnNotNull.class, QueryType.DATA_DEFINITION);

presto-docs/src/main/sphinx/sql/alter-table.rst

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@ Synopsis
1515
ALTER TABLE [ IF EXISTS ] name DROP CONSTRAINT [ IF EXISTS ] constraint_name
1616
ALTER TABLE [ IF EXISTS ] ALTER [ COLUMN ] column_name { SET | DROP } NOT NULL
1717
ALTER TABLE [ IF EXISTS ] name DROP BRANCH [ IF EXISTS ] branch_name
18+
ALTER TABLE [ IF EXISTS ] name DROP TAG [ IF EXISTS ] tag_name
1819
1920
Description
2021
-----------
@@ -94,6 +95,10 @@ Drop branch ``branch1`` from the ``users`` table::
9495

9596
ALTER TABLE users DROP BRANCH 'branch1';
9697

98+
Drop tag ``tag1`` from the ``users`` table::
99+
100+
ALTER TABLE users DROP TAG 'tag1';
101+
97102
See Also
98103
--------
99104

presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -265,6 +265,11 @@ public void checkCanDropBranch(ConnectorTransactionHandle transactionHandle, Con
265265
{
266266
}
267267

268+
@Override
269+
public void checkCanDropTag(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName)
270+
{
271+
}
272+
268273
@Override
269274
public void checkCanDropConstraint(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName)
270275
{

presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -63,6 +63,7 @@
6363
import static com.facebook.presto.spi.security.AccessDeniedException.denyDropRole;
6464
import static com.facebook.presto.spi.security.AccessDeniedException.denyDropSchema;
6565
import static com.facebook.presto.spi.security.AccessDeniedException.denyDropTable;
66+
import static com.facebook.presto.spi.security.AccessDeniedException.denyDropTag;
6667
import static com.facebook.presto.spi.security.AccessDeniedException.denyDropView;
6768
import static com.facebook.presto.spi.security.AccessDeniedException.denyGrantRoles;
6869
import static com.facebook.presto.spi.security.AccessDeniedException.denyGrantTablePrivilege;
@@ -288,6 +289,24 @@ public void checkCanDropBranch(ConnectorTransactionHandle transaction, Connector
288289
}
289290
}
290291

292+
@Override
293+
public void checkCanDropTag(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName)
294+
{
295+
MetastoreContext metastoreContext = new MetastoreContext(
296+
identity, context.getQueryId().getId(),
297+
context.getClientInfo(),
298+
context.getClientTags(),
299+
context.getSource(),
300+
Optional.empty(),
301+
false,
302+
HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER,
303+
context.getWarningCollector(),
304+
context.getRuntimeStats());
305+
if (!isTableOwner(transaction, identity, metastoreContext, tableName)) {
306+
denyDropTag(tableName.toString());
307+
}
308+
}
309+
291310
@Override
292311
public void checkCanDropConstraint(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName)
293312
{

presto-hive/src/main/java/com/facebook/presto/hive/security/SystemTableAwareAccessControl.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -250,6 +250,12 @@ public void checkCanDropBranch(ConnectorTransactionHandle transactionHandle, Con
250250
delegate.checkCanDropBranch(transactionHandle, identity, context, tableName);
251251
}
252252

253+
@Override
254+
public void checkCanDropTag(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName)
255+
{
256+
delegate.checkCanDropTag(transactionHandle, identity, context, tableName);
257+
}
258+
253259
@Override
254260
public void checkCanDropConstraint(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName)
255261
{
Lines changed: 77 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,77 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* http://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
package com.facebook.presto.execution;
15+
16+
import com.facebook.presto.Session;
17+
import com.facebook.presto.common.QualifiedObjectName;
18+
import com.facebook.presto.metadata.Metadata;
19+
import com.facebook.presto.spi.ConnectorId;
20+
import com.facebook.presto.spi.MaterializedViewDefinition;
21+
import com.facebook.presto.spi.PrestoException;
22+
import com.facebook.presto.spi.TableHandle;
23+
import com.facebook.presto.spi.WarningCollector;
24+
import com.facebook.presto.spi.security.AccessControl;
25+
import com.facebook.presto.sql.analyzer.SemanticException;
26+
import com.facebook.presto.sql.tree.DropTag;
27+
import com.facebook.presto.sql.tree.Expression;
28+
import com.facebook.presto.transaction.TransactionManager;
29+
import com.google.common.util.concurrent.ListenableFuture;
30+
31+
import java.util.List;
32+
import java.util.Optional;
33+
34+
import static com.facebook.presto.metadata.MetadataUtil.createQualifiedObjectName;
35+
import static com.facebook.presto.spi.StandardErrorCode.NOT_FOUND;
36+
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.MISSING_TABLE;
37+
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.NOT_SUPPORTED;
38+
import static com.google.common.util.concurrent.Futures.immediateFuture;
39+
40+
public class DropTagTask
41+
implements DDLDefinitionTask<DropTag>
42+
{
43+
@Override
44+
public String getName()
45+
{
46+
return "DROP TAG";
47+
}
48+
49+
@Override
50+
public ListenableFuture<?> execute(DropTag statement, TransactionManager transactionManager, Metadata metadata, AccessControl accessControl, Session session, List<Expression> parameters, WarningCollector warningCollector)
51+
{
52+
QualifiedObjectName tableName = createQualifiedObjectName(session, statement, statement.getTableName());
53+
Optional<TableHandle> tableHandleOptional = metadata.getMetadataResolver(session).getTableHandle(tableName);
54+
55+
if (!tableHandleOptional.isPresent()) {
56+
if (!statement.isTableExists()) {
57+
throw new SemanticException(MISSING_TABLE, statement, "Table '%s' does not exist", tableName);
58+
}
59+
return immediateFuture(null);
60+
}
61+
62+
Optional<MaterializedViewDefinition> optionalMaterializedView = metadata.getMetadataResolver(session).getMaterializedView(tableName);
63+
if (optionalMaterializedView.isPresent()) {
64+
if (!statement.isTableExists()) {
65+
throw new SemanticException(NOT_SUPPORTED, statement, "'%s' is a materialized view, and drop tag is not supported", tableName);
66+
}
67+
return immediateFuture(null);
68+
}
69+
70+
ConnectorId connectorId = metadata.getCatalogHandle(session, tableName.getCatalogName())
71+
.orElseThrow(() -> new PrestoException(NOT_FOUND, "Catalog does not exist: " + tableName.getCatalogName()));
72+
accessControl.checkCanDropTag(session.getRequiredTransactionId(), session.getIdentity(), session.getAccessControlContext(), tableName);
73+
74+
metadata.dropTag(session, tableHandleOptional.get(), Optional.of(statement.getTagName().toString()));
75+
return immediateFuture(null);
76+
}
77+
}

presto-main/src/main/java/com/facebook/presto/metadata/DelegatingMetadataManager.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -637,6 +637,12 @@ public void dropBranch(Session session, TableHandle tableHandle, Optional<String
637637
delegate.dropBranch(session, tableHandle, branchName);
638638
}
639639

640+
@Override
641+
public void dropTag(Session session, TableHandle tableHandle, Optional<String> tagName)
642+
{
643+
delegate.dropTag(session, tableHandle, tagName);
644+
}
645+
640646
@Override
641647
public void dropConstraint(Session session, TableHandle tableHandle, Optional<String> constraintName, Optional<String> columnName)
642648
{

presto-main/src/main/java/com/facebook/presto/metadata/Metadata.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -511,6 +511,8 @@ default TableLayoutFilterCoverage getTableLayoutFilterCoverage(Session session,
511511

512512
void dropBranch(Session session, TableHandle tableHandle, Optional<String> branchName);
513513

514+
void dropTag(Session session, TableHandle tableHandle, Optional<String> tagName);
515+
514516
void dropConstraint(Session session, TableHandle tableHandle, Optional<String> constraintName, Optional<String> columnName);
515517

516518
void addConstraint(Session session, TableHandle tableHandle, TableConstraint<String> tableConstraint);

presto-main/src/main/java/com/facebook/presto/metadata/MetadataManager.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1459,6 +1459,14 @@ public void dropBranch(Session session, TableHandle tableHandle, Optional<String
14591459
metadata.dropBranch(session.toConnectorSession(connectorId), tableHandle.getConnectorHandle(), branchName);
14601460
}
14611461

1462+
@Override
1463+
public void dropTag(Session session, TableHandle tableHandle, Optional<String> tagName)
1464+
{
1465+
ConnectorId connectorId = tableHandle.getConnectorId();
1466+
ConnectorMetadata metadata = getMetadataForWrite(session, connectorId);
1467+
metadata.dropTag(session.toConnectorSession(connectorId), tableHandle.getConnectorHandle(), tagName);
1468+
}
1469+
14621470
@Override
14631471
public void dropConstraint(Session session, TableHandle tableHandle, Optional<String> constraintName, Optional<String> columnName)
14641472
{

presto-main/src/main/java/com/facebook/presto/security/AccessControlManager.java

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -752,6 +752,22 @@ public void checkCanDropBranch(TransactionId transactionId, Identity identity, A
752752
}
753753
}
754754

755+
@Override
756+
public void checkCanDropTag(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName tableName)
757+
{
758+
requireNonNull(identity, "identity is null");
759+
requireNonNull(tableName, "tableName is null");
760+
761+
authenticationCheck(() -> checkCanAccessCatalog(identity, context, tableName.getCatalogName()));
762+
763+
authorizationCheck(() -> systemAccessControl.get().checkCanDropTag(identity, context, toCatalogSchemaTableName(tableName)));
764+
765+
CatalogAccessControlEntry entry = getConnectorAccessControl(transactionId, tableName.getCatalogName());
766+
if (entry != null) {
767+
authorizationCheck(() -> entry.getAccessControl().checkCanDropTag(entry.getTransactionHandle(transactionId), identity.toConnectorIdentity(tableName.getCatalogName()), context, toSchemaTableName(tableName)));
768+
}
769+
}
770+
755771
@Override
756772
public void checkCanDropConstraint(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName tableName)
757773
{

0 commit comments

Comments
 (0)