Skip to content

Commit

Permalink
Add grantor_type and grantee_type columns to table_privileges
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 0ba3120 commit 0beeb77
Show file tree
Hide file tree
Showing 8 changed files with 104 additions and 67 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.listEnabledTablePrivileges;
import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.listEnabledPrincipals;
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 @@ -1843,23 +1843,26 @@ public void revokeTablePrivileges(ConnectorSession session, SchemaTableName sche
@Override
public List<GrantInfo> listTablePrivileges(ConnectorSession session, SchemaTablePrefix schemaTablePrefix)
{
ImmutableList.Builder<GrantInfo> grantInfos = ImmutableList.builder();
Set<PrestoPrincipal> principals = listEnabledPrincipals(metastore, session.getIdentity());
ImmutableList.Builder<GrantInfo> result = ImmutableList.builder();
for (SchemaTableName tableName : listTables(session, schemaTablePrefix)) {
Set<PrivilegeInfo> privileges =
listEnabledTablePrivileges(metastore, tableName.getSchemaName(), tableName.getTableName(), session.getIdentity()).stream()
.map(HivePrivilegeInfo::toPrivilegeInfo)
.flatMap(Set::stream)
.collect(toImmutableSet());

grantInfos.add(
new GrantInfo(
privileges,
session.getIdentity(),
tableName,
Optional.empty(), // Can't access grantor
Optional.empty())); // Can't access withHierarchy
}
return grantInfos.build();
for (PrestoPrincipal grantee : principals) {
Set<HivePrivilegeInfo> hivePrivileges = metastore.listTablePrivileges(tableName.getSchemaName(), tableName.getTableName(), grantee);
for (HivePrivilegeInfo hivePrivilege : hivePrivileges) {
Set<PrivilegeInfo> prestoPrivileges = hivePrivilege.toPrivilegeInfo();
for (PrivilegeInfo prestoPrivilege : prestoPrivileges) {
GrantInfo grant = new GrantInfo(
prestoPrivilege,
grantee,
tableName,
Optional.of(hivePrivilege.getGrantor()),
Optional.empty());
result.add(grant);
}
}
}
}
return result.build();
}

private void verifyJvmTimeZone()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -242,13 +242,18 @@ public static Set<String> listApplicableRoles(SemiTransactionalHiveMetastore met
.collect(toSet());
}

public static Set<HivePrivilegeInfo> listEnabledTablePrivileges(SemiTransactionalHiveMetastore metastore, String databaseName, String tableName, ConnectorIdentity identity)
public static Set<PrestoPrincipal> listEnabledPrincipals(SemiTransactionalHiveMetastore metastore, 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());
return principals.build();
}

public static Set<HivePrivilegeInfo> listEnabledTablePrivileges(SemiTransactionalHiveMetastore metastore, String databaseName, String tableName, ConnectorIdentity identity)
{
return listTablePrivileges(metastore, databaseName, tableName, listEnabledPrincipals(metastore, identity));
}

public static Set<HivePrivilegeInfo> listApplicableTablePrivileges(SemiTransactionalHiveMetastore metastore, String databaseName, String tableName, String user)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,6 @@
import static io.prestosql.metadata.MetadataUtil.TableMetadataBuilder.tableMetadataBuilder;
import static io.prestosql.metadata.MetadataUtil.findColumnMetadata;
import static io.prestosql.spi.type.BigintType.BIGINT;
import static io.prestosql.spi.type.BooleanType.BOOLEAN;
import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType;
import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;
Expand Down Expand Up @@ -109,13 +108,15 @@ public class InformationSchemaMetadata
.build())
.table(tableMetadataBuilder(TABLE_TABLE_PRIVILEGES)
.column("grantor", createUnboundedVarcharType())
.column("grantor_type", createUnboundedVarcharType())
.column("grantee", createUnboundedVarcharType())
.column("grantee_type", createUnboundedVarcharType())
.column("table_catalog", createUnboundedVarcharType())
.column("table_schema", createUnboundedVarcharType())
.column("table_name", createUnboundedVarcharType())
.column("privilege_type", createUnboundedVarcharType())
.column("is_grantable", BOOLEAN)
.column("with_hierarchy", BOOLEAN)
.column("is_grantable", createUnboundedVarcharType())
.column("with_hierarchy", createUnboundedVarcharType())
.build())
.table(tableMetadataBuilder(TABLE_ROLES)
.column("role_name", createUnboundedVarcharType())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@
import io.prestosql.spi.security.AccessDeniedException;
import io.prestosql.spi.security.GrantInfo;
import io.prestosql.spi.security.PrestoPrincipal;
import io.prestosql.spi.security.PrivilegeInfo;
import io.prestosql.spi.security.RoleGrant;

import java.util.ArrayList;
Expand Down Expand Up @@ -196,17 +195,17 @@ private InternalTable buildTablePrivileges(Session session, Set<QualifiedTablePr
for (QualifiedTablePrefix prefix : prefixes) {
List<GrantInfo> grants = ImmutableList.copyOf(listTablePrivileges(session, metadata, accessControl, prefix));
for (GrantInfo grant : grants) {
for (PrivilegeInfo privilegeInfo : grant.getPrivilegeInfo()) {
table.add(
grant.getGrantor().orElse(null),
grant.getIdentity().getUser(),
prefix.getCatalogName(),
grant.getSchemaTableName().getSchemaName(),
grant.getSchemaTableName().getTableName(),
privilegeInfo.getPrivilege().name(),
privilegeInfo.isGrantOption(),
grant.getWithHierarchy().orElse(null));
}
table.add(
grant.getGrantor().map(PrestoPrincipal::getName).orElse(null),
grant.getGrantor().map(principal -> principal.getType().toString()).orElse(null),
grant.getGrantee().getName(),
grant.getGrantee().getType().toString(),
prefix.getCatalogName(),
grant.getSchemaTableName().getSchemaName(),
grant.getSchemaTableName().getTableName(),
grant.getPrivilegeInfo().getPrivilege().name(),
grant.getPrivilegeInfo().isGrantOption() ? "YES" : "NO",
grant.getWithHierarchy().map(withHierarchy -> withHierarchy ? "YES" : "NO").orElse(null));
}
}
return table.build();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -250,15 +250,19 @@ protected Node visitShowGrants(ShowGrants showGrants, Void context)

return simpleQuery(
selectList(
aliasedName("grantor", "Grantor"),
aliasedName("grantor_type", "Grantor Type"),
aliasedName("grantee", "Grantee"),
aliasedName("grantee_type", "Grantee Type"),
aliasedName("table_catalog", "Catalog"),
aliasedName("table_schema", "Schema"),
aliasedName("table_name", "Table"),
aliasedName("privilege_type", "Privilege"),
aliasedName("is_grantable", "Grantable")),
aliasedName("is_grantable", "Grantable"),
aliasedName("with_hierarchy", "With Hierarchy")),
from(catalogName, TABLE_TABLE_PRIVILEGES),
predicate,
Optional.of(ordering(ascending("grantee"), ascending("table_name"))));
Optional.empty());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,13 +14,16 @@
package io.prestosql.tests.hive;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import io.airlift.log.Logger;
import io.prestodb.tempto.AfterTestWithContext;
import io.prestodb.tempto.BeforeTestWithContext;
import io.prestodb.tempto.ProductTest;
import io.prestodb.tempto.query.QueryExecutor;
import org.testng.annotations.Test;

import java.util.Set;

import static io.prestodb.tempto.assertions.QueryAssert.Row;
import static io.prestodb.tempto.assertions.QueryAssert.Row.row;
import static io.prestodb.tempto.assertions.QueryAssert.assertThat;
Expand All @@ -32,10 +35,13 @@
import static io.prestosql.tests.utils.QueryExecutors.onHive;
import static io.prestosql.tests.utils.QueryExecutors.onPresto;
import static java.lang.String.format;
import static java.util.stream.Collectors.toSet;

public class TestGrantRevoke
extends ProductTest
{
private static final Set<String> PREDEFINED_ROLES = ImmutableSet.of("admin", "public");

private String tableName;
private String viewName;
private QueryExecutor aliceExecutor;
Expand Down Expand Up @@ -63,6 +69,7 @@ public void setup()
aliceExecutor.executeQuery(format("CREATE TABLE %s(month bigint, day bigint)", tableName));

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

Expand All @@ -72,12 +79,33 @@ public void cleanup()
try {
aliceExecutor.executeQuery(format("DROP TABLE IF EXISTS %s", tableName));
aliceExecutor.executeQuery(format("DROP VIEW IF EXISTS %s", viewName));
cleanupRoles();
}
catch (Exception e) {
Logger.get(getClass()).warn(e, "failed to drop table/view");
}
}

@Test(groups = {AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
private void cleanupRoles()
{
for (String role : listRoles()) {
if (!PREDEFINED_ROLES.contains(role)) {
onHive().executeQuery(format("DROP ROLE %s", role));
}
}
}

private Set<String> listRoles()
{
return ImmutableSet.copyOf(
onHive().executeQuery("SHOW ROLES")
.rows()
.stream()
.map(row -> row.get(0).toString())
.collect(toSet()));
}

@Test(groups = {AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
public void testGrantRevoke()
{
Expand All @@ -103,20 +131,17 @@ public void testGrantRevoke()
@Test(groups = {AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
public void testShowGrants()
{
assertThat(aliceExecutor.executeQuery(format("SHOW GRANTS ON %s", tableName)))
.containsOnly(ImmutableList.of(
row("alice", "hive", "default", tableName, "SELECT", Boolean.TRUE),
row("alice", "hive", "default", tableName, "INSERT", Boolean.TRUE),
row("alice", "hive", "default", tableName, "UPDATE", Boolean.TRUE),
row("alice", "hive", "default", tableName, "DELETE", Boolean.TRUE)));

onPresto().executeQuery("CREATE ROLE role1");
onPresto().executeQuery(format("GRANT SELECT ON %s TO ROLE role1", tableName));
onPresto().executeQuery("GRANT role1 TO USER bob");
aliceExecutor.executeQuery(format("GRANT SELECT ON %s TO bob WITH GRANT OPTION", tableName));
aliceExecutor.executeQuery(format("GRANT INSERT ON %s TO bob", tableName));

assertThat(bobExecutor.executeQuery(format("SHOW GRANTS ON %s", tableName)))
.containsOnly(ImmutableList.of(
row("bob", "hive", "default", tableName, "SELECT", Boolean.TRUE),
row("bob", "hive", "default", tableName, "INSERT", Boolean.FALSE)));
row("alice", "USER", "bob", "USER", "hive", "default", "alice_owned_table", "SELECT", "YES", null),
row("alice", "USER", "bob", "USER", "hive", "default", "alice_owned_table", "INSERT", "NO", null),
row("hdfs", "USER", "role1", "ROLE", "hive", "default", "alice_owned_table", "SELECT", "NO", null)));
}

@Test(groups = {AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
Expand Down Expand Up @@ -156,7 +181,6 @@ public void testCustomRole()
assertThat(() -> bobExecutor.executeQuery(format("SELECT * FROM %s", tableName)))
.failsWithMessage(format("Access Denied: Cannot select from table default.%s", tableName));
assertThat(aliceExecutor.executeQuery(format("SELECT * FROM %s", tableName))).hasNoRows();
onPresto().executeQuery("DROP ROLE role1");
}

@Test(groups = {AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
Expand All @@ -172,8 +196,6 @@ public void testTransitiveRole()
assertThat(() -> bobExecutor.executeQuery(format("SELECT * FROM %s", tableName)))
.failsWithMessage(format("Access Denied: Cannot select from table default.%s", tableName));
assertThat(aliceExecutor.executeQuery(format("SELECT * FROM %s", tableName))).hasNoRows();
onPresto().executeQuery("DROP ROLE role1");
onPresto().executeQuery("DROP ROLE role2");
}

@Test(groups = {AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,15 @@ system| information_schema| roles| role_name| varchar| YES| null| null|
system| information_schema| schemata| catalog_name| varchar| YES| null| null|
system| information_schema| schemata| schema_name| varchar| YES| null| null|
system| information_schema| table_privileges| grantor| varchar| YES| null| null|
system| information_schema| table_privileges| grantor_type| varchar| YES| null| null|
system| information_schema| table_privileges| grantee| varchar| YES| null| null|
system| information_schema| table_privileges| table_catalog| varchar| YES| null| null |
system| information_schema| table_privileges| table_schema| varchar| YES| null| null |
system| information_schema| table_privileges| grantee_type| varchar| YES| null| null|
system| information_schema| table_privileges| table_catalog| varchar| YES| null| null|
system| information_schema| table_privileges| table_schema| varchar| YES| null| null|
system| information_schema| table_privileges| table_name| varchar| YES| null| null|
system| information_schema| table_privileges| privilege_type| varchar| YES| null| null |
system| information_schema| table_privileges| is_grantable| boolean| YES| null| null |
system| information_schema| table_privileges| with_hierarchy| boolean| YES| null| null |
system| information_schema| table_privileges| privilege_type| varchar| YES| null| null|
system| information_schema| table_privileges| is_grantable| varchar| YES| null| null|
system| information_schema| table_privileges| with_hierarchy| varchar| YES| null| null|
system| information_schema| tables| table_catalog| varchar| YES| null| null|
system| information_schema| tables| table_schema| varchar| YES| null| null|
system| information_schema| tables| table_name| varchar| YES| null| null|
Expand Down
29 changes: 15 additions & 14 deletions presto-spi/src/main/java/io/prestosql/spi/security/GrantInfo.java
Original file line number Diff line number Diff line change
Expand Up @@ -17,31 +17,32 @@

import java.util.Objects;
import java.util.Optional;
import java.util.Set;

import static java.util.Objects.requireNonNull;

public class GrantInfo
{
private final Set<PrivilegeInfo> privilegeInfo;
private final ConnectorIdentity grantee;
private final PrivilegeInfo privilegeInfo;
private final PrestoPrincipal grantee;
private final SchemaTableName schemaTableName;
private final Optional<ConnectorIdentity> grantor;
private final Optional<PrestoPrincipal> grantor;
private final Optional<Boolean> withHierarchy;

public GrantInfo(Set<PrivilegeInfo> privilegeInfo, ConnectorIdentity grantee, SchemaTableName schemaTableName, Optional<ConnectorIdentity> grantor, Optional<Boolean> withHierarchy)
public GrantInfo(PrivilegeInfo privilegeInfo, PrestoPrincipal grantee, SchemaTableName schemaTableName, Optional<PrestoPrincipal> grantor, Optional<Boolean> withHierarchy)
{
this.privilegeInfo = privilegeInfo;
this.grantee = grantee;
this.schemaTableName = schemaTableName;
this.grantor = grantor;
this.withHierarchy = withHierarchy;
this.privilegeInfo = requireNonNull(privilegeInfo, "privilegeInfo is null");
this.grantee = requireNonNull(grantee, "grantee is null");
this.schemaTableName = requireNonNull(schemaTableName, "schemaTableName is null");
this.grantor = requireNonNull(grantor, "grantor is null");
this.withHierarchy = requireNonNull(withHierarchy, "withHierarchy is null");
}

public Set<PrivilegeInfo> getPrivilegeInfo()
public PrivilegeInfo getPrivilegeInfo()
{
return privilegeInfo;
}

public ConnectorIdentity getIdentity()
public PrestoPrincipal getGrantee()
{
return grantee;
}
Expand All @@ -51,7 +52,7 @@ public SchemaTableName getSchemaTableName()
return schemaTableName;
}

public Optional<ConnectorIdentity> getGrantor()
public Optional<PrestoPrincipal> getGrantor()
{
return grantor;
}
Expand All @@ -78,7 +79,7 @@ public boolean equals(Object o)
}
GrantInfo grantInfo = (GrantInfo) o;
return Objects.equals(privilegeInfo, grantInfo.getPrivilegeInfo()) &&
Objects.equals(grantee, grantInfo.getIdentity()) &&
Objects.equals(grantee, grantInfo.getGrantee()) &&
Objects.equals(schemaTableName, grantInfo.getSchemaTableName()) &&
Objects.equals(grantor, grantInfo.getGrantor()) &&
Objects.equals(withHierarchy, grantInfo.getWithHierarchy());
Expand Down

0 comments on commit 0beeb77

Please sign in to comment.