Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
Make ThriftMetastoreUtil.list*Roles methods to return Stream
That way roles are enumerated lazily.
  • Loading branch information
kokosing authored and arhimondr committed Feb 27, 2019
1 parent 881b13c commit 58ac2b7
Show file tree
Hide file tree
Showing 3 changed files with 63 additions and 68 deletions.
Expand Up @@ -1951,13 +1951,15 @@ public void revokeRoles(ConnectorSession session, Set<String> roles, Set<PrestoP
@Override @Override
public Set<RoleGrant> listApplicableRoles(ConnectorSession session, PrestoPrincipal principal) public Set<RoleGrant> listApplicableRoles(ConnectorSession session, PrestoPrincipal principal)
{ {
return ThriftMetastoreUtil.listApplicableRoles(principal, metastore::listRoleGrants); return ThriftMetastoreUtil.listApplicableRoles(principal, metastore::listRoleGrants)
.collect(toImmutableSet());
} }


@Override @Override
public Set<String> listEnabledRoles(ConnectorSession session) public Set<String> listEnabledRoles(ConnectorSession session)
{ {
return ThriftMetastoreUtil.listEnabledRoles(session.getIdentity(), metastore::listRoleGrants); return ThriftMetastoreUtil.listEnabledRoles(session.getIdentity(), metastore::listRoleGrants)
.collect(toImmutableSet());
} }


@Override @Override
Expand Down
Expand Up @@ -39,8 +39,10 @@
import com.facebook.presto.spi.type.MapType; import com.facebook.presto.spi.type.MapType;
import com.facebook.presto.spi.type.RowType; import com.facebook.presto.spi.type.RowType;
import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.Type;
import com.google.common.collect.AbstractIterator;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Streams;
import com.google.common.primitives.Longs; import com.google.common.primitives.Longs;
import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData; import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData; import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
Expand Down Expand Up @@ -81,6 +83,8 @@
import java.util.Queue; import java.util.Queue;
import java.util.Set; import java.util.Set;
import java.util.function.Function; import java.util.function.Function;
import java.util.function.Predicate;
import java.util.stream.Stream;


import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA;
import static com.facebook.presto.hive.HiveMetadata.AVRO_SCHEMA_URL_KEY; import static com.facebook.presto.hive.HiveMetadata.AVRO_SCHEMA_URL_KEY;
Expand Down Expand Up @@ -127,7 +131,6 @@
import static java.util.Locale.ENGLISH; import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull; import static java.util.Objects.requireNonNull;
import static java.util.stream.Collectors.toList; import static java.util.stream.Collectors.toList;
import static java.util.stream.Collectors.toSet;
import static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData.binaryStats; import static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData.binaryStats;
import static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData.booleanStats; import static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData.booleanStats;
import static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData.dateStats; import static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData.dateStats;
Expand Down Expand Up @@ -218,60 +221,66 @@ public static org.apache.hadoop.hive.metastore.api.PrincipalType toMetastoreApiP
} }
} }


public static Set<RoleGrant> listApplicableRoles(PrestoPrincipal principal, Function<PrestoPrincipal, Set<RoleGrant>> listRoleGrants) public static Stream<RoleGrant> listApplicableRoles(PrestoPrincipal principal, Function<PrestoPrincipal, Set<RoleGrant>> listRoleGrants)
{ {
Set<RoleGrant> result = new HashSet<>();
Queue<PrestoPrincipal> queue = new ArrayDeque<>(); Queue<PrestoPrincipal> queue = new ArrayDeque<>();
queue.add(principal); queue.add(principal);
while (!queue.isEmpty()) { Queue<RoleGrant> output = new ArrayDeque<>();
PrestoPrincipal current = queue.poll(); Set<RoleGrant> seenRoles = new HashSet<>();
Set<RoleGrant> grants = listRoleGrants.apply(current); return Streams.stream(new AbstractIterator<RoleGrant>() {
for (RoleGrant grant : grants) { @Override
if (!result.contains(grant)) { protected RoleGrant computeNext()
result.add(grant); {
queue.add(new PrestoPrincipal(ROLE, grant.getRoleName())); if (!output.isEmpty()) {
return output.remove();
} }
if (queue.isEmpty()) {
return endOfData();
}

while (!queue.isEmpty()) {
Set<RoleGrant> grants = listRoleGrants.apply(queue.remove());
if (!grants.isEmpty()) {
for (RoleGrant grant : grants) {
if (seenRoles.add(grant)) {
output.add(grant);
queue.add(new PrestoPrincipal(ROLE, grant.getRoleName()));
}
}
break;
}
}
if (output.isEmpty()) {
return endOfData();
}
return output.remove();
} }
} });
return ImmutableSet.copyOf(result);
} }


public static boolean isRoleApplicable(SemiTransactionalHiveMetastore metastore, PrestoPrincipal principal, String role) public static boolean isRoleApplicable(SemiTransactionalHiveMetastore metastore, PrestoPrincipal principal, String role)
{ {
if (principal.getType() == ROLE && principal.getName().equals(role)) { if (principal.getType() == ROLE && principal.getName().equals(role)) {
return true; return true;
} }
Set<String> seenRoles = new HashSet<>(); return listApplicableRoles(metastore, principal)
Queue<PrestoPrincipal> queue = new ArrayDeque<>(); .anyMatch(role::equals);
queue.add(principal);
while (!queue.isEmpty()) {
Set<RoleGrant> grants = metastore.listRoleGrants(queue.poll());
for (RoleGrant grant : grants) {
if (grant.getRoleName().equals(role)) {
return true;
}
if (seenRoles.add(grant.getRoleName())) {
queue.add(new PrestoPrincipal(ROLE, grant.getRoleName()));
}
}
}
return false;
} }


public static Set<String> listApplicableRoles(SemiTransactionalHiveMetastore metastore, PrestoPrincipal principal) public static Stream<String> listApplicableRoles(SemiTransactionalHiveMetastore metastore, PrestoPrincipal principal)
{ {
return listApplicableRoles(principal, metastore::listRoleGrants) return listApplicableRoles(principal, metastore::listRoleGrants)
.stream() .map(RoleGrant::getRoleName);
.map(RoleGrant::getRoleName)
.collect(toSet());
} }


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


Expand All @@ -285,7 +294,9 @@ public static Set<HivePrivilegeInfo> listApplicableTablePrivileges(SemiTransacti
ImmutableSet.Builder<PrestoPrincipal> principals = ImmutableSet.builder(); ImmutableSet.Builder<PrestoPrincipal> principals = ImmutableSet.builder();
PrestoPrincipal userPrincipal = new PrestoPrincipal(USER, user); PrestoPrincipal userPrincipal = new PrestoPrincipal(USER, user);
principals.add(userPrincipal); principals.add(userPrincipal);
listApplicableRoles(metastore, userPrincipal).stream().map(role -> new PrestoPrincipal(ROLE, role)).forEach(principals::add); listApplicableRoles(metastore, userPrincipal)
.map(role -> new PrestoPrincipal(ROLE, role))
.forEach(principals::add);
return listTablePrivileges(metastore, databaseName, tableName, principals.build()); return listTablePrivileges(metastore, databaseName, tableName, principals.build());
} }


Expand Down Expand Up @@ -325,54 +336,37 @@ public static boolean isRoleEnabled(ConnectorIdentity identity, Function<PrestoP
return false; return false;
} }


Set<String> seenRoles = new HashSet<>(); // all the above code could be removed and method semantic would remain the same, however it would be more expensive for some negative cases (see above)
Queue<PrestoPrincipal> queue = new ArrayDeque<>(); return listEnabledRoles(identity, listRoleGrants)
queue.add(principal); .anyMatch(role::equals);
while (!queue.isEmpty()) {
Set<RoleGrant> grants = listRoleGrants.apply(queue.poll());
for (RoleGrant grant : grants) {
if (grant.getRoleName().equals(role)) {
return true;
}
if (seenRoles.add(grant.getRoleName())) {
queue.add(new PrestoPrincipal(ROLE, grant.getRoleName()));
}
}
}
return false;
} }


public static Set<String> listEnabledRoles(ConnectorIdentity identity, Function<PrestoPrincipal, Set<RoleGrant>> listRoleGrants) public static Stream<String> listEnabledRoles(ConnectorIdentity identity, Function<PrestoPrincipal, Set<RoleGrant>> listRoleGrants)
{ {
Optional<SelectedRole> role = identity.getRole(); Optional<SelectedRole> role = identity.getRole();

if (role.isPresent() && role.get().getType() == SelectedRole.Type.NONE) { if (role.isPresent() && role.get().getType() == SelectedRole.Type.NONE) {
return ImmutableSet.of(PUBLIC_ROLE_NAME); return Stream.of(PUBLIC_ROLE_NAME);
} }

PrestoPrincipal principal; PrestoPrincipal principal;
if (!role.isPresent() || role.get().getType() == SelectedRole.Type.ALL) { if (!role.isPresent() || role.get().getType() == SelectedRole.Type.ALL) {
principal = new PrestoPrincipal(USER, identity.getUser()); principal = new PrestoPrincipal(USER, identity.getUser());
} }
else { else {
principal = new PrestoPrincipal(ROLE, identity.getRole().get().getRole().get()); principal = new PrestoPrincipal(ROLE, role.get().getRole().get());
} }


Set<String> roles = listApplicableRoles(principal, listRoleGrants) Stream<String> roles = Stream.of(PUBLIC_ROLE_NAME);
.stream()
.map(RoleGrant::getRoleName)
.collect(toSet());

// The admin role must be enabled explicitly. If it is, will be re-added below.
roles.remove(ADMIN_ROLE_NAME);

roles.add(PUBLIC_ROLE_NAME);


if (principal.getType() == ROLE) { if (principal.getType() == ROLE) {
roles.add(principal.getName()); roles = Stream.concat(roles, Stream.of(principal.getName()));
} }


return ImmutableSet.copyOf(roles); return Stream.concat(
roles,
listApplicableRoles(principal, listRoleGrants)
.map(RoleGrant::getRoleName)
// The admin role must be enabled explicitly. If it is, it was added above.
.filter(Predicate.isEqual(ADMIN_ROLE_NAME).negate()));
} }


public static org.apache.hadoop.hive.metastore.api.Partition toMetastoreApiPartition(PartitionWithStatistics partitionWithStatistics) public static org.apache.hadoop.hive.metastore.api.Partition toMetastoreApiPartition(PartitionWithStatistics partitionWithStatistics)
Expand Down
Expand Up @@ -430,8 +430,7 @@ private boolean hasAdminOptionForRoles(ConnectorTransactionHandle transaction, C
} }


SemiTransactionalHiveMetastore metastore = metastoreProvider.apply(((HiveTransactionHandle) transaction)); SemiTransactionalHiveMetastore metastore = metastoreProvider.apply(((HiveTransactionHandle) transaction));
Set<RoleGrant> grants = listApplicableRoles(new PrestoPrincipal(USER, identity.getUser()), metastore::listRoleGrants); Set<String> rolesWithGrantOption = listApplicableRoles(new PrestoPrincipal(USER, identity.getUser()), metastore::listRoleGrants)
Set<String> rolesWithGrantOption = grants.stream()
.filter(RoleGrant::isGrantable) .filter(RoleGrant::isGrantable)
.map(RoleGrant::getRoleName) .map(RoleGrant::getRoleName)
.collect(toSet()); .collect(toSet());
Expand Down

0 comments on commit 58ac2b7

Please sign in to comment.