Skip to content

Commit

Permalink
Consider enabled roles for permissions
Browse files Browse the repository at this point in the history
Extracted-From: prestodb/presto#10904
  • Loading branch information
Andrii Rosa authored and sopel39 committed Jan 29, 2019
1 parent 53b13c9 commit c603474
Show file tree
Hide file tree
Showing 10 changed files with 60 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -183,7 +183,7 @@
import static io.prestosql.plugin.hive.metastore.MetastoreUtil.verifyOnline;
import static io.prestosql.plugin.hive.metastore.StorageFormat.VIEW_STORAGE_FORMAT;
import static io.prestosql.plugin.hive.metastore.StorageFormat.fromHiveStorageFormat;
import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.listApplicableTablePrivileges;
import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.listEnabledTablePrivileges;
import static io.prestosql.plugin.hive.util.ConfigurationUtils.toJobConf;
import static io.prestosql.plugin.hive.util.Statistics.ReduceOperator.ADD;
import static io.prestosql.plugin.hive.util.Statistics.createComputedStatisticsToPartitionMap;
Expand Down Expand Up @@ -1845,7 +1845,7 @@ public List<GrantInfo> listTablePrivileges(ConnectorSession session, SchemaTable
ImmutableList.Builder<GrantInfo> grantInfos = ImmutableList.builder();
for (SchemaTableName tableName : listTables(session, schemaTablePrefix)) {
Set<PrivilegeInfo> privileges =
listApplicableTablePrivileges(metastore, tableName.getSchemaName(), tableName.getTableName(), new PrestoPrincipal(USER, session.getUser())).stream()
listEnabledTablePrivileges(metastore, tableName.getSchemaName(), tableName.getTableName(), session.getIdentity()).stream()
.map(HivePrivilegeInfo::toPrivilegeInfo)
.flatMap(Set::stream)
.collect(toImmutableSet());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,6 @@
import java.nio.ByteBuffer;
import java.time.LocalDate;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
Expand Down Expand Up @@ -234,12 +233,26 @@ public static Set<String> listApplicableRoles(SemiTransactionalHiveMetastore met
.collect(toSet());
}

public static Set<HivePrivilegeInfo> listApplicableTablePrivileges(SemiTransactionalHiveMetastore metastore, String databaseName, String tableName, PrestoPrincipal principal)
public static Set<HivePrivilegeInfo> listEnabledTablePrivileges(SemiTransactionalHiveMetastore metastore, String databaseName, String tableName, ConnectorIdentity identity)
{
ImmutableSet.Builder<PrestoPrincipal> principals = ImmutableSet.builder();
PrestoPrincipal userPrincipal = new PrestoPrincipal(USER, identity.getUser());
principals.add(userPrincipal);
listEnabledRoles(identity, metastore::listRoleGrants).stream().map(role -> new PrestoPrincipal(ROLE, role)).forEach(principals::add);
return listTablePrivileges(metastore, databaseName, tableName, principals.build());
}

public static Set<HivePrivilegeInfo> listApplicableTablePrivileges(SemiTransactionalHiveMetastore metastore, String databaseName, String tableName, String user)
{
ImmutableSet.Builder<PrestoPrincipal> principals = ImmutableSet.builder();
PrestoPrincipal userPrincipal = new PrestoPrincipal(USER, user);
principals.add(userPrincipal);
listApplicableRoles(metastore, userPrincipal).stream().map(role -> new PrestoPrincipal(ROLE, role)).forEach(principals::add);
return listTablePrivileges(metastore, databaseName, tableName, principals.build());
}

private static Set<HivePrivilegeInfo> listTablePrivileges(SemiTransactionalHiveMetastore metastore, String databaseName, String tableName, Set<PrestoPrincipal> principals)
{
Set<String> applicableRoles = listApplicableRoles(metastore, principal);
List<PrestoPrincipal> principals = new ArrayList<>();
principals.add(principal);
applicableRoles.stream().map(role -> new PrestoPrincipal(ROLE, role)).forEach(principals::add);
ImmutableSet.Builder<HivePrivilegeInfo> result = ImmutableSet.builder();
for (PrestoPrincipal current : principals) {
result.addAll(metastore.listTablePrivileges(databaseName, tableName, current));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,8 @@
import static io.prestosql.plugin.hive.metastore.HivePrivilegeInfo.toHivePrivilege;
import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.listApplicableRoles;
import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.listApplicableTablePrivileges;
import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.listEnabledRoles;
import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.listEnabledTablePrivileges;
import static io.prestosql.spi.security.AccessDeniedException.denyAddColumn;
import static io.prestosql.spi.security.AccessDeniedException.denyCreateRole;
import static io.prestosql.spi.security.AccessDeniedException.denyCreateSchema;
Expand Down Expand Up @@ -342,7 +344,7 @@ public void checkCanShowRoleGrants(ConnectorTransactionHandle transactionHandle,
private boolean isAdmin(ConnectorTransactionHandle transaction, ConnectorIdentity identity)
{
SemiTransactionalHiveMetastore metastore = metastoreProvider.apply(((HiveTransactionHandle) transaction));
return listApplicableRoles(metastore, new PrestoPrincipal(USER, identity.getUser())).contains(ADMIN_ROLE_NAME);
return listEnabledRoles(identity, metastore::listRoleGrants).contains(ADMIN_ROLE_NAME);
}

private boolean isDatabaseOwner(ConnectorTransactionHandle transaction, ConnectorIdentity identity, String databaseName)
Expand All @@ -368,7 +370,7 @@ private boolean isDatabaseOwner(ConnectorTransactionHandle transaction, Connecto
if (database.getOwnerType() == USER && identity.getUser().equals(database.getOwnerName())) {
return true;
}
if (database.getOwnerType() == ROLE && listApplicableRoles(metastore, new PrestoPrincipal(USER, identity.getUser())).contains(database.getOwnerName())) {
if (database.getOwnerType() == ROLE && listEnabledRoles(identity, metastore::listRoleGrants).contains(database.getOwnerName())) {
return true;
}
return false;
Expand Down Expand Up @@ -399,7 +401,7 @@ private boolean checkTablePermission(
}

SemiTransactionalHiveMetastore metastore = metastoreProvider.apply(((HiveTransactionHandle) transaction));
return listApplicableTablePrivileges(metastore, tableName.getSchemaName(), tableName.getTableName(), new PrestoPrincipal(USER, identity.getUser()))
return listEnabledTablePrivileges(metastore, tableName.getSchemaName(), tableName.getTableName(), identity)
.stream()
.filter(privilegeInfo -> !grantOptionRequired || privilegeInfo.isGrantOption())
.anyMatch(privilegeInfo -> privilegeInfo.getHivePrivilege().equals(requiredPrivilege));
Expand All @@ -416,7 +418,7 @@ private boolean hasGrantOptionForPrivilege(ConnectorTransactionHandle transactio
metastore,
tableName.getSchemaName(),
tableName.getTableName(),
new PrestoPrincipal(USER, identity.getUser()))
identity.getUser())
.contains(new HivePrivilegeInfo(toHivePrivilege(privilege), true));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
package io.prestosql.plugin.hive;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.io.Files;
import io.prestosql.Session;
Expand Down Expand Up @@ -153,15 +154,19 @@ private List<?> getPartitions(HiveTableLayoutHandle tableLayoutHandle)
@Test
public void testSchemaOperations()
{
assertUpdate("CREATE SCHEMA new_schema");
Session admin = Session.builder(getQueryRunner().getDefaultSession())
.setIdentity(new Identity("hive", Optional.empty(), ImmutableMap.of("hive", new SelectedRole(SelectedRole.Type.ROLE, Optional.of("admin")))))
.build();

assertUpdate(admin, "CREATE SCHEMA new_schema");

assertUpdate("CREATE TABLE new_schema.test (x bigint)");
assertUpdate(admin, "CREATE TABLE new_schema.test (x bigint)");

assertQueryFails("DROP SCHEMA new_schema", "Schema not empty: new_schema");
assertQueryFails(admin, "DROP SCHEMA new_schema", "Schema not empty: new_schema");

assertUpdate("DROP TABLE new_schema.test");
assertUpdate(admin, "DROP TABLE new_schema.test");

assertUpdate("DROP SCHEMA new_schema");
assertUpdate(admin, "DROP SCHEMA new_schema");
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -368,13 +368,13 @@ public void testRevokeNonExistingRole()
public void testSetRole()
throws Exception
{
assertUpdate("CREATE ROLE set_role_1");
assertUpdate("CREATE ROLE set_role_2");
assertUpdate("CREATE ROLE set_role_3");
assertUpdate("CREATE ROLE set_role_4");
assertUpdate("GRANT set_role_1 TO USER set_user_1");
assertUpdate("GRANT set_role_2 TO ROLE set_role_1");
assertUpdate("GRANT set_role_3 TO ROLE set_role_2");
executeFromAdmin("CREATE ROLE set_role_1");
executeFromAdmin("CREATE ROLE set_role_2");
executeFromAdmin("CREATE ROLE set_role_3");
executeFromAdmin("CREATE ROLE set_role_4");
executeFromAdmin("GRANT set_role_1 TO USER set_user_1");
executeFromAdmin("GRANT set_role_2 TO ROLE set_role_1");
executeFromAdmin("GRANT set_role_3 TO ROLE set_role_2");

Session unsetRole = Session.builder(getQueryRunner().getDefaultSession())
.setIdentity(new Identity("set_user_1", Optional.empty()))
Expand Down Expand Up @@ -457,10 +457,10 @@ public void testSetRole()

assertQueryFails(setRole4, "SELECT * FROM hive.information_schema.enabled_roles", ".*?Cannot set role set_role_4");

assertUpdate("DROP ROLE set_role_1");
assertUpdate("DROP ROLE set_role_2");
assertUpdate("DROP ROLE set_role_3");
assertUpdate("DROP ROLE set_role_4");
executeFromAdmin("DROP ROLE set_role_1");
executeFromAdmin("DROP ROLE set_role_2");
executeFromAdmin("DROP ROLE set_role_3");
executeFromAdmin("DROP ROLE set_role_4");
}

private Set<String> listRoles()
Expand Down Expand Up @@ -512,7 +512,9 @@ private MaterializedResult executeFromUser(String user, String sql)

private Session createAdminSession()
{
return createUserSession("admin");
return Session.builder(getQueryRunner().getDefaultSession())
.setIdentity(new Identity("admin", Optional.empty(), ImmutableMap.of("hive", new SelectedRole(SelectedRole.Type.ROLE, Optional.of("admin")))))
.build();
}

private Session createUserSession(String user)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ public void setupServer()

try (Connection connection = createConnection();
Statement statement = connection.createStatement()) {
statement.execute("SET ROLE admin");
statement.execute("CREATE SCHEMA default");
statement.execute("CREATE SCHEMA fruit");
}
Expand Down Expand Up @@ -203,7 +204,7 @@ private Connection createConnection(String extra)
throws SQLException
{
String url = format("jdbc:presto://%s/hive/default?%s", server.getAddress(), extra);
return DriverManager.getConnection(url, "test", null);
return DriverManager.getConnection(url, "admin", null);
}

private static Set<String> listTables(Connection connection)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -342,7 +342,7 @@ public interface Metadata
void revokeTablePrivileges(Session session, QualifiedObjectName tableName, Set<Privilege> privileges, PrestoPrincipal grantee, boolean grantOption);

/**
* Gets the privileges for the specified table available to the given grantee
* Gets the privileges for the specified table available to the given grantee considering the selected session role
*/
List<GrantInfo> listTablePrivileges(Session session, QualifiedTablePrefix prefix);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ public void setup()
aliceExecutor.executeQuery(format("DROP TABLE IF EXISTS %s", tableName));
aliceExecutor.executeQuery(format("CREATE TABLE %s(month bigint, day bigint)", tableName));

onPresto().executeQuery("SET ROLE admin");
assertAccessDeniedOnAllOperationsOnTable(bobExecutor, tableName);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ public class TestRoles
@BeforeTestWithContext
public void setUp()
{
onPresto().executeQuery("SET ROLE admin");
onHive().executeQuery("SET ROLE admin");
cleanup();
}
Expand Down Expand Up @@ -481,6 +482,7 @@ public void testSetRole()
@Test(groups = {ROLES, AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
public void testSetAdminRole()
{
onPresto().executeQuery("SET ROLE NONE");
QueryAssert.assertThat(onPresto().executeQuery("SELECT * FROM hive.information_schema.enabled_roles"))
.containsOnly(
row("public"));
Expand All @@ -507,6 +509,7 @@ public void testShowRoles()
QueryAssert.assertThat(() -> onPrestoAlice().executeQuery("SHOW ROLES"))
.failsWithMessage("Cannot show roles from catalog hive");
onPresto().executeQuery("GRANT admin TO alice");
onPrestoAlice().executeQuery("SET ROLE admin");
QueryAssert.assertThat(onPrestoAlice().executeQuery("SHOW ROLES"))
.containsOnly(
row("public"),
Expand All @@ -517,7 +520,7 @@ public void testShowRoles()
@Test(groups = {ROLES, AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
public void testShowCurrentRoles()
{
QueryAssert.assertThat(onPresto().executeQuery("SHOW CURRENT ROLES"))
QueryAssert.assertThat(onPrestoAlice().executeQuery("SHOW CURRENT ROLES"))
.containsOnly(
row("public"));
onPresto().executeQuery("CREATE ROLE role1");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -517,7 +517,7 @@ default void revokeTablePrivileges(ConnectorSession session, SchemaTableName tab
}

/**
* List the table privileges granted to the specified grantee for the tables that have the specified prefix
* List the table privileges granted to the specified grantee for the tables that have the specified prefix considering the selected session role
*/
default List<GrantInfo> listTablePrivileges(ConnectorSession session, SchemaTablePrefix prefix)
{
Expand Down

0 comments on commit c603474

Please sign in to comment.