Skip to content

Commit

Permalink
Support TRUNCATE TABLE statement
Browse files Browse the repository at this point in the history
Cherry-pick of trinodb/trino#8932

Co-authored-by: Yuya Ebihara <yuya.ebihara@starburstdata.com>
  • Loading branch information
2 people authored and rschlussel committed Mar 30, 2022
1 parent 3b4b5e3 commit c34dad7
Show file tree
Hide file tree
Showing 41 changed files with 467 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -545,6 +545,21 @@ public void dropTable(JdbcIdentity identity, JdbcTableHandle handle)
}
}

@Override
public void truncateTable(JdbcIdentity identity, JdbcTableHandle jdbcTableHandle)
{
StringBuilder sql = new StringBuilder()
.append("TRUNCATE TABLE ")
.append(quoted(jdbcTableHandle.getCatalogName(), jdbcTableHandle.getSchemaName(), jdbcTableHandle.getTableName()));

try (Connection connection = connectionFactory.openConnection(identity)) {
execute(connection, sql.toString());
}
catch (SQLException e) {
throw new PrestoException(JDBC_ERROR, e);
}
}

@Override
public void rollbackCreateTable(JdbcIdentity identity, JdbcOutputTableHandle handle)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,8 @@ PreparedStatement buildSql(ConnectorSession session, Connection connection, Jdbc

void dropTable(JdbcIdentity identity, JdbcTableHandle jdbcTableHandle);

void truncateTable(JdbcIdentity identity, JdbcTableHandle jdbcTableHandle);

void rollbackCreateTable(JdbcIdentity identity, JdbcOutputTableHandle handle);

String buildInsertSql(JdbcOutputTableHandle handle);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -259,6 +259,11 @@ public List<ConnectorTableLayoutResult> getTableLayouts(
return ImmutableList.of(new ConnectorTableLayoutResult(getTableLayout(session, layoutHandle), constraint.getSummary()));
}

@Override
public void truncateTable(ConnectorSession session, ConnectorTableHandle tableHandle)
{
}

@Override
public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTableLayoutHandle handle)
{
Expand Down
1 change: 1 addition & 0 deletions presto-docs/src/main/sphinx/sql.rst
Original file line number Diff line number Diff line change
Expand Up @@ -57,5 +57,6 @@ This chapter describes the SQL syntax used in Presto.
sql/show-stats
sql/show-tables
sql/start-transaction
sql/truncate
sql/use
sql/values
28 changes: 28 additions & 0 deletions presto-docs/src/main/sphinx/sql/truncate.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
========
TRUNCATE
========

Synopsis
--------

.. code-block:: none
TRUNCATE TABLE table_name
Description
-----------

Delete all rows from a table.

Examples
--------

Truncate the table ``orders``::

TRUNCATE TABLE orders;

Limitations
-----------

Some connectors have limited or no support for ``TRUNCATE``.
See connector documentation for more details.
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,11 @@ public void checkCanRenameTable(ConnectorTransactionHandle transaction, Connecto
}
}

@Override
public void checkCanTruncateTable(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName)
{
}

@Override
public void checkCanShowTablesMetadata(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, String schemaName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@
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.denyShowRoles;
import static com.facebook.presto.spi.security.AccessDeniedException.denyTruncateTable;
import static com.facebook.presto.spi.security.PrincipalType.ROLE;
import static com.facebook.presto.spi.security.PrincipalType.USER;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
Expand Down Expand Up @@ -229,6 +230,15 @@ public void checkCanDeleteFromTable(ConnectorTransactionHandle transaction, Conn
}
}

@Override
public void checkCanTruncateTable(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName)
{
MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER);
if (!checkTablePermission(transaction, identity, metastoreContext, tableName, DELETE, false)) {
denyTruncateTable(tableName.toString());
}
}

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

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_TABLE;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.NOT_SUPPORTED;
import static com.google.common.util.concurrent.Futures.immediateFuture;

public class TruncateTableTask
implements DDLDefinitionTask<TruncateTable>
{
@Override
public String getName()
{
return "TRUNCATE TABLE";
}

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

if (metadata.isMaterializedView(session, tableName)) {
throw new SemanticException(NOT_SUPPORTED, statement, "Cannot truncate a materialized view");
}

if (metadata.isView(session, tableName)) {
throw new SemanticException(NOT_SUPPORTED, statement, "Cannot truncate a view");
}

Optional<TableHandle> tableHandle = metadata.getTableHandle(session, tableName);
if (!tableHandle.isPresent()) {
throw new SemanticException(MISSING_TABLE, statement, "Table '%s' does not exist", tableName);
}

accessControl.checkCanTruncateTable(session.getRequiredTransactionId(), session.getIdentity(), session.getAccessControlContext(), tableName);

metadata.truncateTable(session, tableHandle.get());

return immediateFuture(null);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -233,6 +233,11 @@ public interface Metadata
*/
void dropTable(Session session, TableHandle tableHandle);

/**
* Truncates the specified table
*/
void truncateTable(Session session, TableHandle tableHandle);

Optional<NewTableLayout> getNewTableLayout(Session session, String catalogName, ConnectorTableMetadata tableMetadata);

@Experimental
Expand Down Expand Up @@ -348,6 +353,14 @@ public interface Metadata
*/
Optional<ViewDefinition> getView(Session session, QualifiedObjectName viewName);

/**
* Is the specified table a view.
*/
default boolean isView(Session session, QualifiedObjectName viewName)
{
return getView(session, viewName).isPresent();
}

/**
* Creates the specified view with the specified view definition.
*/
Expand All @@ -363,6 +376,14 @@ public interface Metadata
*/
Optional<ConnectorMaterializedViewDefinition> getMaterializedView(Session session, QualifiedObjectName viewName);

/**
* Is the specified table a materialized view.
*/
default boolean isMaterializedView(Session session, QualifiedObjectName viewName)
{
return getMaterializedView(session, viewName).isPresent();
}

/**
* Creates the specified materialized view with the specified view definition.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -725,6 +725,14 @@ public void dropTable(Session session, TableHandle tableHandle)
metadata.dropTable(session.toConnectorSession(connectorId), tableHandle.getConnectorHandle());
}

@Override
public void truncateTable(Session session, TableHandle tableHandle)
{
ConnectorId connectorId = tableHandle.getConnectorId();
ConnectorMetadata metadata = getMetadataForWrite(session, connectorId);
metadata.truncateTable(session.toConnectorSession(connectorId), tableHandle.getConnectorHandle());
}

@Override
public Optional<NewTableLayout> getInsertLayout(Session session, TableHandle table)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -161,6 +161,13 @@ public interface AccessControl
*/
void checkCanDeleteFromTable(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName tableName);

/**
* Check if identity is allowed to truncate the specified table.
*
* @throws com.facebook.presto.spi.security.AccessDeniedException if not allowed
*/
void checkCanTruncateTable(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName tableName);

/**
* Check if identity is allowed to create the specified view.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -440,6 +440,22 @@ public void checkCanDeleteFromTable(TransactionId transactionId, Identity identi
}
}

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

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

authorizationCheck(() -> systemAccessControl.get().checkCanTruncateTable(identity, context, toCatalogSchemaTableName(tableName)));

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

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

@Override
public void checkCanTruncateTable(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName tableName)
{
}

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

@Override
public void checkCanTruncateTable(Identity identity, AccessControlContext context, CatalogSchemaTableName table)
{
}

@Override
public void checkCanCreateView(Identity identity, AccessControlContext context, CatalogSchemaTableName view)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@
import static com.facebook.presto.spi.security.AccessDeniedException.denyShowRoles;
import static com.facebook.presto.spi.security.AccessDeniedException.denyShowSchemas;
import static com.facebook.presto.spi.security.AccessDeniedException.denyShowTablesMetadata;
import static com.facebook.presto.spi.security.AccessDeniedException.denyTruncateTable;
import static com.google.common.collect.ImmutableSet.toImmutableSet;

public class DenyAllAccessControl
Expand Down Expand Up @@ -179,6 +180,12 @@ public void checkCanDeleteFromTable(TransactionId transactionId, Identity identi
denyDeleteTable(tableName.toString());
}

@Override
public void checkCanTruncateTable(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName tableName)
{
denyTruncateTable(tableName.toString());
}

@Override
public void checkCanCreateView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@
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.denySetUser;
import static com.facebook.presto.spi.security.AccessDeniedException.denyTruncateTable;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Suppliers.memoizeWithExpiration;
import static java.lang.String.format;
Expand Down Expand Up @@ -276,6 +277,14 @@ public void checkCanDropTable(Identity identity, AccessControlContext context, C
}
}

@Override
public void checkCanTruncateTable(Identity identity, AccessControlContext context, CatalogSchemaTableName table)
{
if (!canAccessCatalog(identity, table.getCatalogName(), ALL)) {
denyTruncateTable(table.toString());
}
}

@Override
public void checkCanRenameTable(Identity identity, AccessControlContext context, CatalogSchemaTableName table, CatalogSchemaTableName newTable)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,7 @@
import com.facebook.presto.sql.tree.Statement;
import com.facebook.presto.sql.tree.Table;
import com.facebook.presto.sql.tree.TableSubquery;
import com.facebook.presto.sql.tree.TruncateTable;
import com.facebook.presto.sql.tree.Union;
import com.facebook.presto.sql.tree.Unnest;
import com.facebook.presto.sql.tree.Use;
Expand Down Expand Up @@ -904,6 +905,12 @@ protected Scope visitProperty(Property node, Optional<Scope> scope)
return createAndAssignScope(node, scope);
}

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

@Override
protected Scope visitDropTable(DropTable node, Optional<Scope> scope)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@
import com.facebook.presto.execution.StartTransactionTask;
import com.facebook.presto.execution.TaskManagerConfig;
import com.facebook.presto.execution.TaskSource;
import com.facebook.presto.execution.TruncateTableTask;
import com.facebook.presto.execution.resourceGroups.NoOpResourceGroupManager;
import com.facebook.presto.execution.scheduler.LegacyNetworkTopology;
import com.facebook.presto.execution.scheduler.NodeScheduler;
Expand Down Expand Up @@ -191,6 +192,7 @@
import com.facebook.presto.sql.tree.SetSession;
import com.facebook.presto.sql.tree.StartTransaction;
import com.facebook.presto.sql.tree.Statement;
import com.facebook.presto.sql.tree.TruncateTable;
import com.facebook.presto.testing.PageConsumerOperator.PageConsumerOutputFactory;
import com.facebook.presto.tracing.TracingConfig;
import com.facebook.presto.transaction.InMemoryTransactionManager;
Expand Down Expand Up @@ -499,6 +501,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig,
.put(DropFunction.class, new DropFunctionTask(sqlParser))
.put(DropTable.class, new DropTableTask())
.put(DropView.class, new DropViewTask())
.put(TruncateTable.class, new TruncateTableTask())
.put(DropMaterializedView.class, new DropMaterializedViewTask())
.put(RenameColumn.class, new RenameColumnTask())
.put(RenameTable.class, new RenameTableTask())
Expand Down
Loading

0 comments on commit c34dad7

Please sign in to comment.