Skip to content

Commit

Permalink
Add test for grant and revoke on table
Browse files Browse the repository at this point in the history
  • Loading branch information
lukasz-walkiewicz authored and kokosing committed Nov 2, 2020
1 parent 4e083b1 commit 652697c
Show file tree
Hide file tree
Showing 5 changed files with 365 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -294,5 +294,17 @@ public void revokeSchemaPrivileges(ConnectorSession session, String schemaName,
{
accessControl.revokeSchemaPrivileges(schemaName, privileges, revokee, grantOption);
}

@Override
public void grantTablePrivileges(ConnectorSession session, SchemaTableName tableName, Set<Privilege> privileges, PrestoPrincipal grantee, boolean grantOption)
{
accessControl.grantTablePrivileges(tableName, privileges, grantee, grantOption);
}

@Override
public void revokeTablePrivileges(ConnectorSession session, SchemaTableName tableName, Set<Privilege> privileges, PrestoPrincipal revokee, boolean grantOption)
{
accessControl.revokeTablePrivileges(tableName, privileges, revokee, grantOption);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

import io.prestosql.plugin.base.security.AllowAllAccessControl;
import io.prestosql.spi.connector.ConnectorSecurityContext;
import io.prestosql.spi.connector.SchemaTableName;
import io.prestosql.spi.security.ConnectorIdentity;
import io.prestosql.spi.security.PrestoPrincipal;
import io.prestosql.spi.security.Privilege;
Expand All @@ -24,7 +25,9 @@

import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static io.prestosql.spi.security.AccessDeniedException.denyGrantSchemaPrivilege;
import static io.prestosql.spi.security.AccessDeniedException.denyGrantTablePrivilege;
import static io.prestosql.spi.security.AccessDeniedException.denyRevokeSchemaPrivilege;
import static io.prestosql.spi.security.AccessDeniedException.denyRevokeTablePrivilege;
import static java.util.Objects.requireNonNull;

class MockConnectorAccessControl
Expand All @@ -33,10 +36,12 @@ class MockConnectorAccessControl
private static final String INFORMATION_SCHEMA = "information_schema";

private final Grants<String> schemaGrants;
private final Grants<SchemaTableName> tableGrants;

MockConnectorAccessControl(Grants<String> schemaGrants)
MockConnectorAccessControl(Grants<String> schemaGrants, Grants<SchemaTableName> tableGrants)
{
this.schemaGrants = requireNonNull(schemaGrants, "schemaGrants is null");
this.tableGrants = requireNonNull(tableGrants, "tableGrants is null");
}

@Override
Expand All @@ -63,6 +68,32 @@ public void checkCanRevokeSchemaPrivilege(ConnectorSecurityContext context, Priv
}
}

@Override
public Set<SchemaTableName> filterTables(ConnectorSecurityContext context, Set<SchemaTableName> tableNames)
{
return tableNames.stream()
.filter(tableName -> canAccessSchema(context.getIdentity(), tableName.getSchemaName()) || canAccessTable(context.getIdentity(), tableName))
.collect(toImmutableSet());
}

@Override
public void checkCanGrantTablePrivilege(ConnectorSecurityContext context, Privilege privilege, SchemaTableName tableName, PrestoPrincipal grantee, boolean grantOption)
{
String user = context.getIdentity().getUser();
if (!schemaGrants.canGrant(user, tableName.getSchemaName(), privilege) && !tableGrants.canGrant(user, tableName, privilege)) {
denyGrantTablePrivilege(privilege.toString(), tableName.getTableName());
}
}

@Override
public void checkCanRevokeTablePrivilege(ConnectorSecurityContext context, Privilege privilege, SchemaTableName tableName, PrestoPrincipal revokee, boolean grantOption)
{
String user = context.getIdentity().getUser();
if (!schemaGrants.canGrant(user, tableName.getSchemaName(), privilege) && !tableGrants.canGrant(user, tableName, privilege)) {
denyRevokeTablePrivilege(privilege.toString(), tableName.toString());
}
}

public void grantSchemaPrivileges(String schemaName, Set<Privilege> privileges, PrestoPrincipal grantee, boolean grantOption)
{
schemaGrants.grant(grantee, schemaName, privileges, grantOption);
Expand All @@ -73,9 +104,24 @@ public void revokeSchemaPrivileges(String schemaName, Set<Privilege> privileges,
schemaGrants.revoke(revokee, schemaName, privileges, grantOption);
}

private boolean canAccessSchema(ConnectorIdentity identity, String schema)
public void grantTablePrivileges(SchemaTableName tableName, Set<Privilege> privileges, PrestoPrincipal grantee, boolean grantOption)
{
tableGrants.grant(grantee, tableName, privileges, grantOption);
}

public void revokeTablePrivileges(SchemaTableName tableName, Set<Privilege> privileges, PrestoPrincipal revokee, boolean grantOption)
{
tableGrants.revoke(revokee, tableName, privileges, grantOption);
}

private boolean canAccessSchema(ConnectorIdentity identity, String schemaName)
{
return schemaName.equalsIgnoreCase(INFORMATION_SCHEMA)
|| Arrays.stream(Privilege.values()).anyMatch(privilege -> schemaGrants.isAllowed(identity.getUser(), schemaName, privilege));
}

private boolean canAccessTable(ConnectorIdentity identity, SchemaTableName tableName)
{
return schema.equalsIgnoreCase(INFORMATION_SCHEMA)
|| Arrays.stream(Privilege.values()).anyMatch(privilege -> schemaGrants.isAllowed(identity.getUser(), schema, privilege));
return Arrays.stream(Privilege.values()).anyMatch(privilege -> tableGrants.isAllowed(identity.getUser(), tableName, privilege));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -161,6 +161,7 @@ public static final class Builder
private ListRoleGrants roleGrants = defaultRoleAuthorizations();
private ApplyTopN applyTopN = (session, handle, topNCount, sortItems, assignments) -> Optional.empty();
private Grants<String> schemaGrants = new AllowAllGrants<>();
private Grants<SchemaTableName> tableGrants = new AllowAllGrants<>();

public Builder withListSchemaNames(Function<ConnectorSession, List<String>> listSchemaNames)
{
Expand Down Expand Up @@ -244,6 +245,12 @@ public Builder withSchemaGrants(Grants<String> schemaGrants)
return this;
}

public Builder withTableGrants(Grants<SchemaTableName> tableGrants)
{
this.tableGrants = tableGrants;
return this;
}

public MockConnectorFactory build()
{
return new MockConnectorFactory(
Expand All @@ -258,7 +265,7 @@ public MockConnectorFactory build()
getNewTableLayout,
eventListeners,
roleGrants,
new MockConnectorAccessControl(schemaGrants));
new MockConnectorAccessControl(schemaGrants, tableGrants));
}

public static Function<ConnectorSession, List<String>> defaultListSchemaNames()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,150 @@
/*
* 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 io.prestosql.execution;

import com.google.common.collect.ImmutableList;
import io.prestosql.Session;
import io.prestosql.connector.Grants;
import io.prestosql.connector.MockConnectorFactory;
import io.prestosql.connector.MockConnectorPlugin;
import io.prestosql.connector.MockConnectorTableHandle;
import io.prestosql.connector.MutableGrants;
import io.prestosql.spi.connector.SchemaTableName;
import io.prestosql.spi.security.Identity;
import io.prestosql.spi.security.PrestoPrincipal;
import io.prestosql.spi.security.Privilege;
import io.prestosql.sql.query.QueryAssertions;
import io.prestosql.testing.DataProviders;
import io.prestosql.testing.DistributedQueryRunner;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;

import java.util.EnumSet;

import static io.prestosql.common.Randoms.randomUsername;
import static io.prestosql.spi.security.PrincipalType.USER;
import static io.prestosql.testing.TestingSession.testSessionBuilder;
import static java.lang.String.format;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;

public class TestGrantOnTable
{
private final SchemaTableName table = new SchemaTableName("default", "table_one");
private final Session admin = sessionOf("admin");
private final Grants<SchemaTableName> tableGrants = new MutableGrants<>();
private DistributedQueryRunner queryRunner;
private QueryAssertions assertions;

@BeforeClass
public void initClass()
throws Exception
{
queryRunner = DistributedQueryRunner.builder(admin).build();
MockConnectorFactory connectorFactory = MockConnectorFactory.builder()
.withListSchemaNames(session -> ImmutableList.of("default"))
.withListTables((session, schemaName) -> "default".equalsIgnoreCase(schemaName) ? ImmutableList.of(table) : ImmutableList.of())
.withGetTableHandle((session, tableName) -> tableName.equals(table) ? new MockConnectorTableHandle(tableName) : null)
.withSchemaGrants(new MutableGrants<>())
.withTableGrants(tableGrants)
.build();
queryRunner.installPlugin(new MockConnectorPlugin(connectorFactory));
queryRunner.createCatalog("local", "mock");
assertions = new QueryAssertions(queryRunner);
tableGrants.grant(new PrestoPrincipal(USER, "admin"), table, EnumSet.allOf(Privilege.class), true);
}

@AfterClass(alwaysRun = true)
public void teardown()
{
assertions.close();
assertions = null;
}

@Test(dataProviderClass = DataProviders.class, dataProvider = "trueFalse")
public void testExistingGrants(boolean grantOption)
{
String username = randomUsername();
Session user = sessionOf(username);
tableGrants.grant(new PrestoPrincipal(USER, username), table, EnumSet.allOf(Privilege.class), grantOption);

assertThat(assertions.query(admin, "SHOW TABLES FROM local.default")).matches("VALUES (VARCHAR 'table_one')");
assertThat(assertions.query(user, "SHOW TABLES FROM local.default")).matches("VALUES (VARCHAR 'table_one')");
}

@Test(dataProvider = "privileges")
public void testValidGrant(String privilege)
{
String username = randomUsername();
Session user = sessionOf(username);

queryRunner.execute(admin, format("GRANT %s ON TABLE table_one TO %s", privilege, username));

assertThat(assertions.query(user, "SHOW TABLES FROM default")).matches("VALUES (VARCHAR 'table_one')");
}

@Test(dataProvider = "privileges")
public void testValidGrantWithGrantOption(String privilege)
{
String username = randomUsername();
Session user = sessionOf(username);

queryRunner.execute(admin, format("GRANT %s ON TABLE table_one TO %s WITH GRANT OPTION", privilege, username));

assertThat(assertions.query(user, "SHOW TABLES FROM default")).matches("VALUES (VARCHAR 'table_one')");
assertThat(assertions.query(user, format("GRANT %s ON TABLE table_one TO %s", privilege, randomUsername()))).matches("VALUES (BOOLEAN 'TRUE')");
assertThat(assertions.query(user, format("GRANT %s ON TABLE table_one TO %s WITH GRANT OPTION", privilege, randomUsername()))).matches("VALUES (BOOLEAN 'TRUE')");
}

@Test(dataProvider = "privileges")
public void testGrantOnNonExistingCatalog(String privilege)
{
assertThatThrownBy(() -> queryRunner.execute(admin, format("GRANT %s ON TABLE missing_catalog.missing_schema.missing_table TO %s", privilege, randomUsername())))
.hasMessageContaining("Table 'missing_catalog.missing_schema.missing_table' does not exist");
}

@Test(dataProvider = "privileges")
public void testGrantOnNonExistingSchema(String privilege)
{
assertThatThrownBy(() -> queryRunner.execute(admin, format("GRANT %s ON TABLE missing_schema.missing_table TO %s", privilege, randomUsername())))
.hasMessageContaining("Table 'local.missing_schema.missing_table' does not exist");
}

@Test(dataProvider = "privileges")
public void testGrantOnNonExistingTable(String privilege)
{
assertThatThrownBy(() -> queryRunner.execute(admin, format("GRANT %s ON TABLE default.missing_table TO %s", privilege, randomUsername())))
.hasMessageContaining("Table 'local.default.missing_table' does not exist");
}

@DataProvider(name = "privileges")
public static Object[][] privileges()
{
return new Object[][] {
{"SELECT"},
{"ALL PRIVILEGES"}
};
}

private static Session sessionOf(String username)
{
return testSessionBuilder()
.setIdentity(Identity.ofUser(username))
.setCatalog("local")
.setSchema("default")
.build();
}
}

0 comments on commit 652697c

Please sign in to comment.