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
  • Loading branch information
Andrii Rosa authored and Andrii Rosa committed Mar 17, 2017
1 parent 2944b74 commit e4cddc0
Show file tree
Hide file tree
Showing 7 changed files with 96 additions and 58 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@
import static com.facebook.presto.hive.HiveWriteUtils.isWritableType;
import static com.facebook.presto.hive.metastore.HivePrivilegeInfo.toHivePrivilege;
import static com.facebook.presto.hive.metastore.MetastoreUtil.getHiveSchema;
import static com.facebook.presto.hive.metastore.MetastoreUtil.listEnabledTablePrivileges;
import static com.facebook.presto.hive.metastore.MetastoreUtil.listEnabledPrincipals;
import static com.facebook.presto.hive.metastore.SemiTransactionalHiveMetastore.WriteMode.DIRECT_TO_TARGET_EXISTING_DIRECTORY;
import static com.facebook.presto.hive.metastore.SemiTransactionalHiveMetastore.WriteMode.DIRECT_TO_TARGET_NEW_DIRECTORY;
import static com.facebook.presto.hive.metastore.SemiTransactionalHiveMetastore.WriteMode.STAGE_AND_MOVE_TO_TARGET_DIRECTORY;
Expand Down Expand Up @@ -1306,24 +1306,26 @@ public void revokeTablePrivileges(ConnectorSession session, SchemaTableName sche
@Override
public List<GrantInfo> listTablePrivileges(ConnectorSession session, SchemaTablePrefix schemaTablePrefix)
{
ImmutableList.Builder<GrantInfo> grantInfoBuilder = ImmutableList.builder();
Set<PrestoPrincipal> principals = listEnabledPrincipals(metastore, session.getIdentity());
ImmutableList.Builder<GrantInfo> result = ImmutableList.builder();
for (SchemaTableName tableName : listTables(session, schemaTablePrefix)) {
Set<PrivilegeInfo> privilegeInfoSet =
listEnabledTablePrivileges(metastore, tableName.getSchemaName(), tableName.getTableName(), session.getIdentity())
.stream()
.map(HivePrivilegeInfo::toPrivilegeInfo)
.flatMap(Collection::stream)
.collect(toSet());

grantInfoBuilder.add(
new GrantInfo(
privilegeInfoSet,
session.getIdentity(),
tableName,
Optional.empty(), // Can't access grantor
Optional.empty())); // Can't access withHierarchy
}
return grantInfoBuilder.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();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -416,13 +416,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 @@ -38,7 +38,6 @@
import static com.facebook.presto.metadata.MetadataUtil.TableMetadataBuilder.tableMetadataBuilder;
import static com.facebook.presto.metadata.MetadataUtil.findColumnMetadata;
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.spi.type.BooleanType.BOOLEAN;
import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Predicates.compose;
Expand Down Expand Up @@ -102,13 +101,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 @@ -43,7 +43,6 @@
import com.facebook.presto.spi.predicate.TupleDomain;
import com.facebook.presto.spi.security.GrantInfo;
import com.facebook.presto.spi.security.PrestoPrincipal;
import com.facebook.presto.spi.security.PrivilegeInfo;
import com.facebook.presto.spi.security.RoleGrant;
import com.facebook.presto.transaction.TransactionManager;
import com.google.common.base.Throwables;
Expand Down Expand Up @@ -234,17 +233,17 @@ private InternalTable buildTablePrivileges(Session session, String catalogName,
List<GrantInfo> grants = ImmutableList.copyOf(listTablePrivileges(session, metadata, accessControl, prefix));
InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_TABLE_PRIVILEGES));
for (GrantInfo grant : grants) {
for (PrivilegeInfo privilegeInfo : grant.getPrivilegeInfo()) {
table.add(
grant.getGrantor().orElse(null),
grant.getIdentity().getUser(),
catalogName,
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(),
catalogName,
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 @@ -234,18 +234,21 @@ protected Node visitShowGrants(ShowGrants showGrant, Void context)
session.getRequiredTransactionId(),
session.getIdentity(),
new QualifiedTablePrefix(catalogName, session.getSchema(), optionalTableName));

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 @@ -15,13 +15,16 @@
package com.facebook.presto.tests.hive;

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

import java.util.Set;

import static com.facebook.presto.tests.TestGroups.AUTHORIZATION;
import static com.facebook.presto.tests.TestGroups.HIVE_CONNECTOR;
import static com.facebook.presto.tests.TestGroups.PROFILE_SPECIFIC_TESTS;
Expand All @@ -34,10 +37,13 @@
import static com.teradata.tempto.context.ThreadLocalTestContextHolder.testContext;
import static com.teradata.tempto.sql.SqlContexts.createViewAs;
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 @@ -65,6 +71,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 @@ -74,12 +81,32 @@ 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");
}
}

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 = {HIVE_CONNECTOR, AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
public void testGrantRevoke()
{
Expand All @@ -105,14 +132,17 @@ public void testGrantRevoke()
@Test(groups = {HIVE_CONNECTOR, AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
public void testShowGrants()
{
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", "alice_owned_table", "SELECT", Boolean.TRUE),
row("bob", "hive", "default", "alice_owned_table", "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 = {HIVE_CONNECTOR, AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
Expand Down Expand Up @@ -153,7 +183,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 = {HIVE_CONNECTOR, AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
Expand All @@ -170,8 +199,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 = {HIVE_CONNECTOR, AUTHORIZATION, PROFILE_SPECIFIC_TESTS})
Expand Down
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 e4cddc0

Please sign in to comment.