Skip to content

Commit

Permalink
Check SET PROPERTIES permission in CREATE TABLE task
Browse files Browse the repository at this point in the history
  • Loading branch information
ebyhr committed Oct 14, 2021
1 parent 12210d5 commit 2f25d78
Show file tree
Hide file tree
Showing 24 changed files with 198 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import io.trino.spi.security.AccessDeniedException;
import io.trino.spi.type.Type;
import io.trino.spi.type.TypeNotFoundException;
import io.trino.sql.analyzer.FeaturesConfig;
import io.trino.sql.analyzer.Output;
import io.trino.sql.analyzer.OutputColumn;
import io.trino.sql.tree.ColumnDefinition;
Expand All @@ -44,6 +45,8 @@
import io.trino.sql.tree.TableElement;
import io.trino.transaction.TransactionManager;

import javax.inject.Inject;

import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
Expand Down Expand Up @@ -81,6 +84,14 @@
public class CreateTableTask
implements DataDefinitionTask<CreateTable>
{
private final boolean disableSetPropertiesSecurityCheckForCreateDdl;

@Inject
public CreateTableTask(FeaturesConfig featuresConfig)
{
this.disableSetPropertiesSecurityCheckForCreateDdl = featuresConfig.isDisableSetPropertiesSecurityCheckForCreateDdl();
}

@Override
public String getName()
{
Expand Down Expand Up @@ -231,8 +242,6 @@ else if (element instanceof LikeClause) {
}
}

accessControl.checkCanCreateTable(session.toSecurityContext(), tableName);

Map<String, Expression> sqlProperties = mapFromProperties(statement.getProperties());
Map<String, Object> properties = metadata.getTablePropertyManager().getProperties(
catalogName,
Expand All @@ -244,6 +253,13 @@ else if (element instanceof LikeClause) {
parameterLookup,
true);

if (!disableSetPropertiesSecurityCheckForCreateDdl && !properties.isEmpty()) {
accessControl.checkCanCreateTable(session.toSecurityContext(), tableName, properties);
}
else {
accessControl.checkCanCreateTable(session.toSecurityContext(), tableName);
}

Map<String, Object> finalProperties = combineProperties(sqlProperties.keySet(), properties, inheritedProperties);

ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(tableName.asSchemaTableName(), ImmutableList.copyOf(columns.values()), finalProperties, statement.getComment());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,9 +164,18 @@ public interface AccessControl
* Check if identity is allowed to create the specified table.
*
* @throws AccessDeniedException if not allowed
* @deprecated use {@link #checkCanCreateTable(SecurityContext context, QualifiedObjectName tableName, Map properties)}
*/
@Deprecated
void checkCanCreateTable(SecurityContext context, QualifiedObjectName tableName);

/**
* Check if identity is allowed to create the specified table with properties.
*
* @throws AccessDeniedException if not allowed
*/
void checkCanCreateTable(SecurityContext context, QualifiedObjectName tableName, Map<String, Object> properties);

/**
* Check if identity is allowed to drop the specified table.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -426,6 +426,19 @@ public void checkCanCreateTable(SecurityContext securityContext, QualifiedObject
catalogAuthorizationCheck(tableName.getCatalogName(), securityContext, (control, context) -> control.checkCanCreateTable(context, tableName.asSchemaTableName()));
}

@Override
public void checkCanCreateTable(SecurityContext securityContext, QualifiedObjectName tableName, Map<String, Object> properties)
{
requireNonNull(securityContext, "securityContext is null");
requireNonNull(tableName, "tableName is null");

checkCanAccessCatalog(securityContext, tableName.getCatalogName());

systemAuthorizationCheck(control -> control.checkCanCreateTable(securityContext.toSystemSecurityContext(), tableName.asCatalogSchemaTableName(), properties));

catalogAuthorizationCheck(tableName.getCatalogName(), securityContext, (control, context) -> control.checkCanCreateTable(context, tableName.asSchemaTableName(), properties));
}

@Override
public void checkCanDropTable(SecurityContext securityContext, QualifiedObjectName tableName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,11 @@ public void checkCanCreateTable(SecurityContext context, QualifiedObjectName tab
{
}

@Override
public void checkCanCreateTable(SecurityContext context, QualifiedObjectName tableName, Map<String, Object> properties)
{
}

@Override
public void checkCanDropTable(SecurityContext context, QualifiedObjectName tableName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,12 @@ public void checkCanCreateTable(SecurityContext context, QualifiedObjectName tab
denyCreateTable(tableName.toString());
}

@Override
public void checkCanCreateTable(SecurityContext context, QualifiedObjectName tableName, Map<String, Object> properties)
{
denyCreateTable(tableName.toString());
}

@Override
public void checkCanDropTable(SecurityContext context, QualifiedObjectName tableName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -159,6 +159,12 @@ public void checkCanCreateTable(SecurityContext context, QualifiedObjectName tab
delegate().checkCanCreateTable(context, tableName);
}

@Override
public void checkCanCreateTable(SecurityContext context, QualifiedObjectName tableName, Map<String, Object> properties)
{
delegate().checkCanCreateTable(context, tableName, properties);
}

@Override
public void checkCanDropTable(SecurityContext context, QualifiedObjectName tableName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,13 @@ public void checkCanCreateTable(ConnectorSecurityContext context, SchemaTableNam
accessControl.checkCanCreateTable(securityContext, getQualifiedObjectName(tableName));
}

@Override
public void checkCanCreateTable(ConnectorSecurityContext context, SchemaTableName tableName, Map<String, Object> properties)
{
checkArgument(context == null, "context must be null");
accessControl.checkCanCreateTable(securityContext, getQualifiedObjectName(tableName), properties);
}

@Override
public void checkCanDropTable(ConnectorSecurityContext context, SchemaTableName tableName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -141,6 +141,7 @@ public class FeaturesConfig
private int maxGroupingSets = 2048;

private boolean legacyCatalogRoles;
private boolean disableSetPropertiesSecurityCheckForCreateDdl;

public enum JoinReorderingStrategy
{
Expand Down Expand Up @@ -1090,4 +1091,16 @@ public FeaturesConfig setLegacyCatalogRoles(boolean legacyCatalogRoles)
this.legacyCatalogRoles = legacyCatalogRoles;
return this;
}

public boolean isDisableSetPropertiesSecurityCheckForCreateDdl()
{
return disableSetPropertiesSecurityCheckForCreateDdl;
}

@Config("deprecated.disable-set-properties-security-check-for-create-ddl")
public FeaturesConfig setDisableSetPropertiesSecurityCheckForCreateDdl(boolean disableSetPropertiesSecurityCheckForCreateDdl)
{
this.disableSetPropertiesSecurityCheckForCreateDdl = disableSetPropertiesSecurityCheckForCreateDdl;
return this;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -451,7 +451,7 @@ private LocalQueryRunner(
defaultSession.getProtocolHeaders());

dataDefinitionTask = ImmutableMap.<Class<? extends Statement>, DataDefinitionTask<?>>builder()
.put(CreateTable.class, new CreateTableTask())
.put(CreateTable.class, new CreateTableTask(featuresConfig))
.put(CreateView.class, new CreateViewTask(sqlParser, groupProvider, statsCalculator))
.put(DropTable.class, new DropTableTask())
.put(DropView.class, new DropViewTask())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -340,6 +340,20 @@ public void checkCanCreateTable(SecurityContext context, QualifiedObjectName tab
}
}

@Override
public void checkCanCreateTable(SecurityContext context, QualifiedObjectName tableName, Map<String, Object> properties)
{
if (shouldDenyPrivilege(context.getIdentity().getUser(), tableName.getObjectName(), CREATE_TABLE)) {
denyCreateTable(tableName.toString());
}
if (shouldDenyPrivilege(context.getIdentity().getUser(), tableName.getObjectName(), SET_TABLE_PROPERTIES)) {
denySetTableProperties(tableName.toString());
}
if (denyPrivileges.isEmpty()) {
super.checkCanCreateTable(context, tableName, properties);
}
}

@Override
public void checkCanDropTable(SecurityContext context, QualifiedObjectName tableName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import io.trino.spi.type.Type;
import io.trino.spi.type.TypeId;
import io.trino.spi.type.TypeSignature;
import io.trino.sql.analyzer.FeaturesConfig;
import io.trino.sql.planner.TestingConnectorTransactionHandle;
import io.trino.sql.tree.ColumnDefinition;
import io.trino.sql.tree.CreateTable;
Expand Down Expand Up @@ -144,7 +145,7 @@ public void testCreateTableNotExistsTrue()
ImmutableList.of(),
Optional.empty());

getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {}));
getFutureValue(new CreateTableTask(new FeaturesConfig()).internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {}));
assertEquals(metadata.getCreateTableCallCount(), 1);
}

Expand All @@ -157,7 +158,7 @@ public void testCreateTableNotExistsFalse()
ImmutableList.of(),
Optional.empty());

assertTrinoExceptionThrownBy(() -> getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {})))
assertTrinoExceptionThrownBy(() -> getFutureValue(new CreateTableTask(new FeaturesConfig()).internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {})))
.hasErrorCode(ALREADY_EXISTS)
.hasMessage("Table already exists");

Expand All @@ -173,7 +174,7 @@ public void testCreateTableWithMaterializedViewPropertyFails()
ImmutableList.of(new Property(new Identifier("foo"), new StringLiteral("bar"))),
Optional.empty());

assertTrinoExceptionThrownBy(() -> getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {})))
assertTrinoExceptionThrownBy(() -> getFutureValue(new CreateTableTask(new FeaturesConfig()).internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {})))
.hasErrorCode(INVALID_TABLE_PROPERTY)
.hasMessage("Catalog 'catalog' does not support table property 'foo'");

Expand All @@ -190,7 +191,7 @@ public void testCreateWithNotNullColumns()
new ColumnDefinition(identifier("c"), toSqlType(VARBINARY), false, emptyList(), Optional.empty()));
CreateTable statement = new CreateTable(QualifiedName.of("test_table"), inputColumns, true, ImmutableList.of(), Optional.empty());

getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {}));
getFutureValue(new CreateTableTask(new FeaturesConfig()).internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {}));
assertEquals(metadata.getCreateTableCallCount(), 1);
List<ColumnMetadata> columns = metadata.getReceivedTableMetadata().get(0).getColumns();
assertEquals(columns.size(), 3);
Expand Down Expand Up @@ -223,7 +224,7 @@ public void testCreateWithUnsupportedConnectorThrowsWhenNotNull()
Optional.empty());

assertTrinoExceptionThrownBy(() ->
getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {})))
getFutureValue(new CreateTableTask(new FeaturesConfig()).internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {})))
.hasErrorCode(NOT_SUPPORTED)
.hasMessage("Catalog 'catalog' does not support non-null column for column name 'b'");
}
Expand All @@ -233,7 +234,7 @@ public void testCreateLike()
{
CreateTable statement = getCreatleLikeStatement(false);

getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, List.of(), output -> {}));
getFutureValue(new CreateTableTask(new FeaturesConfig()).internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, List.of(), output -> {}));
assertEquals(metadata.getCreateTableCallCount(), 1);

assertThat(metadata.getReceivedTableMetadata().get(0).getColumns())
Expand All @@ -246,7 +247,7 @@ public void testCreateLikeWithProperties()
{
CreateTable statement = getCreatleLikeStatement(true);

getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, List.of(), output -> {}));
getFutureValue(new CreateTableTask(new FeaturesConfig()).internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, List.of(), output -> {}));
assertEquals(metadata.getCreateTableCallCount(), 1);

assertThat(metadata.getReceivedTableMetadata().get(0).getColumns())
Expand All @@ -263,7 +264,7 @@ public void testCreateLikeDenyPermission()
TestingAccessControlManager accessControl = new TestingAccessControlManager(transactionManager, new EventListenerManager(new EventListenerConfig()));
accessControl.deny(privilege("parent_table", SELECT_COLUMN));

assertThatThrownBy(() -> getFutureValue(new CreateTableTask().internalExecute(statement, metadata, accessControl, testSession, List.of(), output -> {})))
assertThatThrownBy(() -> getFutureValue(new CreateTableTask(new FeaturesConfig()).internalExecute(statement, metadata, accessControl, testSession, List.of(), output -> {})))
.isInstanceOf(AccessDeniedException.class)
.hasMessageContaining("Cannot reference columns of table");
}
Expand All @@ -276,7 +277,7 @@ public void testCreateLikeWithPropertiesDenyPermission()
TestingAccessControlManager accessControl = new TestingAccessControlManager(transactionManager, new EventListenerManager(new EventListenerConfig()));
accessControl.deny(privilege("parent_table", SHOW_CREATE_TABLE));

assertThatThrownBy(() -> getFutureValue(new CreateTableTask().internalExecute(statement, metadata, accessControl, testSession, List.of(), output -> {})))
assertThatThrownBy(() -> getFutureValue(new CreateTableTask(new FeaturesConfig()).internalExecute(statement, metadata, accessControl, testSession, List.of(), output -> {})))
.isInstanceOf(AccessDeniedException.class)
.hasMessageContaining("Cannot reference properties of table");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,8 @@ public void testDefaults()
.setUseTableScanNodePartitioning(true)
.setTableScanNodePartitioningMinBucketToTaskRatio(0.5)
.setMergeProjectWithValues(true)
.setLegacyCatalogRoles(false));
.setLegacyCatalogRoles(false)
.setDisableSetPropertiesSecurityCheckForCreateDdl(false));
}

@Test
Expand Down Expand Up @@ -192,6 +193,7 @@ public void testExplicitPropertyMappings()
.put("optimizer.table-scan-node-partitioning-min-bucket-to-task-ratio", "0.0")
.put("optimizer.merge-project-with-values", "false")
.put("deprecated.legacy-catalog-roles", "true")
.put("deprecated.disable-set-properties-security-check-for-create-ddl", "true")
.build();

FeaturesConfig expected = new FeaturesConfig()
Expand Down Expand Up @@ -266,7 +268,8 @@ public void testExplicitPropertyMappings()
.setUseTableScanNodePartitioning(false)
.setTableScanNodePartitioningMinBucketToTaskRatio(0.0)
.setMergeProjectWithValues(false)
.setLegacyCatalogRoles(true);
.setLegacyCatalogRoles(true)
.setDisableSetPropertiesSecurityCheckForCreateDdl(true);
assertFullMapping(properties, expected);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -159,12 +159,24 @@ default void checkCanShowCreateTable(ConnectorSecurityContext context, SchemaTab
* Check if identity is allowed to create the specified table.
*
* @throws io.trino.spi.security.AccessDeniedException if not allowed
* @deprecated use {@link #checkCanCreateTable(ConnectorSecurityContext context, SchemaTableName tableName, Map properties)} instead
*/
@Deprecated
default void checkCanCreateTable(ConnectorSecurityContext context, SchemaTableName tableName)
{
denyCreateTable(tableName.toString());
}

/**
* Check if identity is allowed to create the specified table with properties.
*
* @throws io.trino.spi.security.AccessDeniedException if not allowed
*/
default void checkCanCreateTable(ConnectorSecurityContext context, SchemaTableName tableName, Map<String, Object> properties)
{
denyCreateTable(tableName.toString());
}

/**
* Check if identity is allowed to drop the specified table.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -285,12 +285,24 @@ default void checkCanShowCreateTable(SystemSecurityContext context, CatalogSchem
* Check if identity is allowed to create the specified table in a catalog.
*
* @throws AccessDeniedException if not allowed
* @deprecated use {@link #checkCanCreateTable(SystemSecurityContext context, CatalogSchemaTableName table, Map properties)} instead
*/
@Deprecated
default void checkCanCreateTable(SystemSecurityContext context, CatalogSchemaTableName table)
{
denyCreateTable(table.toString());
}

/**
* Check if identity is allowed to create the specified table with properties in a catalog.
*
* @throws AccessDeniedException if not allowed
*/
default void checkCanCreateTable(SystemSecurityContext context, CatalogSchemaTableName table, Map<String, Object> properties)
{
denyCreateTable(table.toString());
}

/**
* Check if identity is allowed to drop the specified table in a catalog.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,14 @@ public void checkCanCreateTable(ConnectorSecurityContext context, SchemaTableNam
}
}

@Override
public void checkCanCreateTable(ConnectorSecurityContext context, SchemaTableName tableName, Map<String, Object> properties)
{
try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) {
delegate.checkCanCreateTable(context, tableName, properties);
}
}

@Override
public void checkCanDropTable(ConnectorSecurityContext context, SchemaTableName tableName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,11 @@ public void checkCanCreateTable(ConnectorSecurityContext context, SchemaTableNam
{
}

@Override
public void checkCanCreateTable(ConnectorSecurityContext context, SchemaTableName tableName, Map<String, Object> properties)
{
}

@Override
public void checkCanDropTable(ConnectorSecurityContext context, SchemaTableName tableName)
{
Expand Down
Loading

0 comments on commit 2f25d78

Please sign in to comment.