From cb2da7bd91f05c880533d256952a09e57cf5a540 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Tue, 28 Nov 2017 14:50:47 +0100 Subject: [PATCH] Refactor GRANT/REVOKE in Hive Leverage newly introduced method for recursive role grants traversal Extracted-From: https://github.com/prestodb/presto/pull/10904 --- .../prestosql/plugin/hive/HiveMetadata.java | 13 +- .../hive/metastore/CachingHiveMetastore.java | 63 ++----- .../hive/metastore/ExtendedHiveMetastore.java | 8 +- .../metastore/RecordingHiveMetastore.java | 67 ++----- .../SemiTransactionalHiveMetastore.java | 50 +---- .../plugin/hive/metastore/UserTableKey.java | 19 +- .../metastore/file/FileHiveMetastore.java | 79 +------- .../metastore/glue/GlueHiveMetastore.java | 27 +-- .../thrift/BridgingHiveMetastore.java | 32 +--- .../hive/metastore/thrift/HiveMetastore.java | 38 +--- .../metastore/thrift/HiveMetastoreClient.java | 4 - .../metastore/thrift/ThriftHiveMetastore.java | 177 +++++------------- .../thrift/ThriftHiveMetastoreClient.java | 8 - .../metastore/thrift/ThriftMetastoreUtil.java | 65 +++++-- .../security/SqlStandardAccessControl.java | 63 ++++++- .../metastore/TestRecordingHiveMetastore.java | 24 +-- .../metastore/UnimplementedHiveMetastore.java | 14 +- .../thrift/InMemoryHiveMetastore.java | 96 +--------- .../thrift/MockHiveMetastoreClient.java | 7 - 19 files changed, 246 insertions(+), 608 deletions(-) diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveMetadata.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveMetadata.java index ef4af677811f..20bd14cd8da5 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveMetadata.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveMetadata.java @@ -40,6 +40,7 @@ import io.prestosql.plugin.hive.metastore.SortingColumn; import io.prestosql.plugin.hive.metastore.StorageFormat; import io.prestosql.plugin.hive.metastore.Table; +import io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil; import io.prestosql.plugin.hive.statistics.HiveStatisticsProvider; import io.prestosql.spi.PrestoException; import io.prestosql.spi.StandardErrorCode; @@ -182,6 +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.util.ConfigurationUtils.toJobConf; import static io.prestosql.plugin.hive.util.Statistics.ReduceOperator.ADD; import static io.prestosql.plugin.hive.util.Statistics.createComputedStatisticsToPartitionMap; @@ -1796,7 +1798,7 @@ public void revokeRoles(ConnectorSession session, Set roles, Set listApplicableRoles(ConnectorSession session, PrestoPrincipal principal) { - return metastore.listApplicableRoles(principal); + return ThriftMetastoreUtil.listApplicableRoles(principal, metastore::listRoleGrants); } @Override @@ -1830,10 +1832,11 @@ public List listTablePrivileges(ConnectorSession session, SchemaTable { ImmutableList.Builder grantInfos = ImmutableList.builder(); for (SchemaTableName tableName : listTables(session, schemaTablePrefix)) { - Set privileges = metastore.getTablePrivileges(session.getUser(), tableName.getSchemaName(), tableName.getTableName()).stream() - .map(HivePrivilegeInfo::toPrivilegeInfo) - .flatMap(Set::stream) - .collect(toImmutableSet()); + Set privileges = + listApplicableTablePrivileges(metastore, tableName.getSchemaName(), tableName.getTableName(), new PrestoPrincipal(USER, session.getUser())).stream() + .map(HivePrivilegeInfo::toPrivilegeInfo) + .flatMap(Set::stream) + .collect(toImmutableSet()); grantInfos.add( new GrantInfo( diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/CachingHiveMetastore.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/CachingHiveMetastore.java index ed9057de12e3..6027a74d2531 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/CachingHiveMetastore.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/CachingHiveMetastore.java @@ -62,6 +62,7 @@ import static io.prestosql.plugin.hive.metastore.HivePartitionName.hivePartitionName; import static io.prestosql.plugin.hive.metastore.HiveTableName.hiveTableName; import static io.prestosql.plugin.hive.metastore.PartitionFilter.partitionFilter; +import static io.prestosql.spi.security.PrincipalType.USER; import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -83,8 +84,7 @@ public class CachingHiveMetastore private final LoadingCache> partitionCache; private final LoadingCache>> partitionFilterCache; private final LoadingCache>> partitionNamesCache; - private final LoadingCache> userRolesCache; - private final LoadingCache> userTablePrivileges; + private final LoadingCache> tablePrivilegesCache; private final LoadingCache> rolesCache; private final LoadingCache> roleGrantsCache; @@ -187,11 +187,8 @@ public Map> loadAll(Iterable loadRoles(user)), executor)); - - userTablePrivileges = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) - .build(asyncReloading(CacheLoader.from(key -> loadTablePrivileges(key.getUser(), key.getDatabase(), key.getTable())), executor)); + tablePrivilegesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) + .build(asyncReloading(CacheLoader.from(key -> loadTablePrivileges(key.getDatabase(), key.getTable(), key.getPrincipal())), executor)); rolesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) .build(asyncReloading(CacheLoader.from(() -> loadRoles()), executor)); @@ -211,10 +208,9 @@ public void flushCache() tableCache.invalidateAll(); partitionCache.invalidateAll(); partitionFilterCache.invalidateAll(); - userTablePrivileges.invalidateAll(); + tablePrivilegesCache.invalidateAll(); tableStatisticsCache.invalidateAll(); partitionStatisticsCache.invalidateAll(); - userRolesCache.invalidateAll(); rolesCache.invalidateAll(); } @@ -504,9 +500,9 @@ protected void invalidateTable(String databaseName, String tableName) tableCache.invalidate(hiveTableName(databaseName, tableName)); tableNamesCache.invalidate(databaseName); viewNamesCache.invalidate(databaseName); - userTablePrivileges.asMap().keySet().stream() + tablePrivilegesCache.asMap().keySet().stream() .filter(userTableKey -> userTableKey.matches(databaseName, tableName)) - .forEach(userTablePrivileges::invalidate); + .forEach(tablePrivilegesCache::invalidate); tableStatisticsCache.invalidate(hiveTableName(databaseName, tableName)); invalidatePartitionCache(databaseName, tableName); } @@ -631,7 +627,6 @@ public void createRole(String role, String grantor) } finally { rolesCache.invalidateAll(); - userRolesCache.invalidate(grantor); } } @@ -643,7 +638,6 @@ public void dropRole(String role) } finally { rolesCache.invalidateAll(); - userRolesCache.invalidateAll(); roleGrantsCache.invalidateAll(); } } @@ -707,34 +701,6 @@ private void invalidatePartitionCache(String databaseName, String tableName) .forEach(partitionStatisticsCache::invalidate); } - @Override - public Set getRoles(String user) - { - return get(userRolesCache, user); - } - - private Set loadRoles(String user) - { - return delegate.getRoles(user); - } - - @Override - public Set getDatabasePrivileges(String user, String databaseName) - { - return delegate.getDatabasePrivileges(user, databaseName); - } - - @Override - public Set getTablePrivileges(String user, String databaseName, String tableName) - { - return get(userTablePrivileges, new UserTableKey(user, tableName, databaseName)); - } - - private Set loadTablePrivileges(String user, String databaseName, String tableName) - { - return delegate.getTablePrivileges(user, databaseName, tableName); - } - @Override public void grantTablePrivileges(String databaseName, String tableName, String grantee, Set privileges) { @@ -742,7 +708,7 @@ public void grantTablePrivileges(String databaseName, String tableName, String g delegate.grantTablePrivileges(databaseName, tableName, grantee, privileges); } finally { - userTablePrivileges.invalidate(new UserTableKey(grantee, tableName, databaseName)); + tablePrivilegesCache.invalidate(new UserTableKey(new PrestoPrincipal(USER, grantee), databaseName, tableName)); } } @@ -753,10 +719,21 @@ public void revokeTablePrivileges(String databaseName, String tableName, String delegate.revokeTablePrivileges(databaseName, tableName, grantee, privileges); } finally { - userTablePrivileges.invalidate(new UserTableKey(grantee, tableName, databaseName)); + tablePrivilegesCache.invalidate(new UserTableKey(new PrestoPrincipal(USER, grantee), databaseName, tableName)); } } + @Override + public Set listTablePrivileges(String databaseName, String tableName, PrestoPrincipal principal) + { + return get(tablePrivilegesCache, new UserTableKey(principal, databaseName, tableName)); + } + + public Set loadTablePrivileges(String databaseName, String tableName, PrestoPrincipal principal) + { + return delegate.listTablePrivileges(databaseName, tableName, principal); + } + private static CacheBuilder newCacheBuilder(OptionalLong expiresAfterWriteMillis, OptionalLong refreshMillis, long maximumSize) { CacheBuilder cacheBuilder = CacheBuilder.newBuilder(); diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/ExtendedHiveMetastore.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/ExtendedHiveMetastore.java index 3e0c1e365d25..fed6765b133a 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/ExtendedHiveMetastore.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/ExtendedHiveMetastore.java @@ -99,13 +99,9 @@ public interface ExtendedHiveMetastore Set listRoleGrants(PrestoPrincipal principal); - Set getRoles(String user); - - Set getDatabasePrivileges(String user, String databaseName); - - Set getTablePrivileges(String user, String databaseName, String tableName); - void grantTablePrivileges(String databaseName, String tableName, String grantee, Set privileges); void revokeTablePrivileges(String databaseName, String tableName, String grantee, Set privileges); + + Set listTablePrivileges(String databaseName, String tableName, PrestoPrincipal principal); } diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/RecordingHiveMetastore.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/RecordingHiveMetastore.java index 7f8a1b11579f..45fb7ff0753a 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/RecordingHiveMetastore.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/RecordingHiveMetastore.java @@ -74,9 +74,7 @@ public class RecordingHiveMetastore private final Cache>> partitionNamesCache; private final Cache>> partitionNamesByPartsCache; private final Cache, Map>> partitionsByNamesCache; - private final Cache> rolesCache; - private final Cache> databasePrivilegesCache; - private final Cache> tablePrivilegesCache; + private final Cache> listTablePrivilegesCache; private final Cache> listRolesCache; private final Cache> listRoleGrantsCache; @@ -100,9 +98,7 @@ public RecordingHiveMetastore(@ForRecordingHiveMetastore ExtendedHiveMetastore d partitionNamesCache = createCache(hiveClientConfig); partitionNamesByPartsCache = createCache(hiveClientConfig); partitionsByNamesCache = createCache(hiveClientConfig); - rolesCache = createCache(hiveClientConfig); - databasePrivilegesCache = createCache(hiveClientConfig); - tablePrivilegesCache = createCache(hiveClientConfig); + listTablePrivilegesCache = createCache(hiveClientConfig); listRolesCache = createCache(hiveClientConfig); listRoleGrantsCache = createCache(hiveClientConfig); @@ -129,9 +125,7 @@ void loadRecording() partitionNamesCache.putAll(toMap(recording.getPartitionNames())); partitionNamesByPartsCache.putAll(toMap(recording.getPartitionNamesByParts())); partitionsByNamesCache.putAll(toMap(recording.getPartitionsByNames())); - rolesCache.putAll(toMap(recording.getRoles())); - databasePrivilegesCache.putAll(toMap(recording.getDatabasePrivileges())); - tablePrivilegesCache.putAll(toMap(recording.getTablePrivileges())); + listTablePrivilegesCache.putAll(toMap(recording.getListTablePrivileges())); listRolesCache.putAll(toMap(recording.getListRoles())); listRoleGrantsCache.putAll(toMap(recording.getListRoleGrants())); } @@ -169,9 +163,7 @@ public void writeRecording() toPairs(partitionNamesCache), toPairs(partitionNamesByPartsCache), toPairs(partitionsByNamesCache), - toPairs(rolesCache), - toPairs(databasePrivilegesCache), - toPairs(tablePrivilegesCache), + toPairs(listTablePrivilegesCache), toPairs(listRolesCache), toPairs(listRoleGrantsCache)); new ObjectMapperProvider().get() @@ -394,27 +386,12 @@ public void alterPartition(String databaseName, String tableName, PartitionWithS } @Override - public Set getRoles(String user) - { - return loadValue(rolesCache, user, () -> delegate.getRoles(user)); - } - - @Override - public Set getDatabasePrivileges(String user, String databaseName) - { - return loadValue( - databasePrivilegesCache, - new UserDatabaseKey(user, databaseName), - () -> delegate.getDatabasePrivileges(user, databaseName)); - } - - @Override - public Set getTablePrivileges(String user, String databaseName, String tableName) + public Set listTablePrivileges(String databaseName, String tableName, PrestoPrincipal principal) { return loadValue( - tablePrivilegesCache, - new UserTableKey(user, databaseName, tableName), - () -> delegate.getTablePrivileges(user, databaseName, tableName)); + listTablePrivilegesCache, + new UserTableKey(principal, databaseName, tableName), + () -> delegate.listTablePrivileges(databaseName, tableName, principal)); } @Override @@ -518,9 +495,7 @@ public static class Recording private final List>>> partitionNames; private final List>>> partitionNamesByParts; private final List, Map>>> partitionsByNames; - private final List>> roles; - private final List>> databasePrivileges; - private final List>> tablePrivileges; + private final List>> listTablePrivileges; private final List>> listRoles; private final List>> listRoleGrants; @@ -538,9 +513,7 @@ public Recording( @JsonProperty("partitionNames") List>>> partitionNames, @JsonProperty("partitionNamesByParts") List>>> partitionNamesByParts, @JsonProperty("partitionsByNames") List, Map>>> partitionsByNames, - @JsonProperty("roles") List>> roles, - @JsonProperty("databasePrivileges") List>> databasePrivileges, - @JsonProperty("tablePrivileges") List>> tablePrivileges, + @JsonProperty("listTablePrivileges") List>> listTablePrivileges, @JsonProperty("listRoles") List>> listRoles, @JsonProperty("listRoleGrants") List>> listRoleGrants) { @@ -556,9 +529,7 @@ public Recording( this.partitionNames = partitionNames; this.partitionNamesByParts = partitionNamesByParts; this.partitionsByNames = partitionsByNames; - this.roles = roles; - this.databasePrivileges = databasePrivileges; - this.tablePrivileges = tablePrivileges; + this.listTablePrivileges = listTablePrivileges; this.listRoles = listRoles; this.listRoleGrants = listRoleGrants; } @@ -636,21 +607,9 @@ public List, Map>>> getP } @JsonProperty - public List>> getRoles() - { - return roles; - } - - @JsonProperty - public List>> getDatabasePrivileges() - { - return databasePrivileges; - } - - @JsonProperty - public List>> getTablePrivileges() + public List>> getListTablePrivileges() { - return tablePrivileges; + return listTablePrivileges; } @JsonProperty diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/SemiTransactionalHiveMetastore.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/SemiTransactionalHiveMetastore.java index 252fa2b71b24..a9dd61454dbc 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/SemiTransactionalHiveMetastore.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/SemiTransactionalHiveMetastore.java @@ -35,6 +35,7 @@ import io.prestosql.spi.connector.SchemaTableName; import io.prestosql.spi.connector.TableNotFoundException; import io.prestosql.spi.security.PrestoPrincipal; +import io.prestosql.spi.security.PrincipalType; import io.prestosql.spi.security.RoleGrant; import io.prestosql.spi.statistics.ColumnStatisticType; import io.prestosql.spi.type.Type; @@ -50,11 +51,9 @@ import java.util.Collection; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Queue; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -83,7 +82,6 @@ import static io.prestosql.spi.StandardErrorCode.ALREADY_EXISTS; import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; import static io.prestosql.spi.StandardErrorCode.TRANSACTION_CONFLICT; -import static io.prestosql.spi.security.PrincipalType.ROLE; import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static org.apache.hadoop.hive.common.FileUtils.makePartName; @@ -764,65 +762,37 @@ public synchronized void revokeRoles(Set roles, Set gra setExclusive((delegate, hdfsEnvironment) -> delegate.revokeRoles(roles, grantees, adminOptionFor, grantor)); } - public synchronized Set listApplicableRoles(PrestoPrincipal principal) - { - checkReadable(); - Set result = new HashSet<>(); - Queue queue = new LinkedList<>(); - queue.add(principal); - while (!queue.isEmpty()) { - PrestoPrincipal current = queue.poll(); - Set grants = listRoleGrants(current); - for (RoleGrant grant : grants) { - if (!result.contains(grant)) { - result.add(grant); - queue.add(new PrestoPrincipal(ROLE, grant.getRoleName())); - } - } - } - return ImmutableSet.copyOf(result); - } - public synchronized Set listRoleGrants(PrestoPrincipal principal) { checkReadable(); return delegate.listRoleGrants(principal); } - public synchronized Set getRoles(String user) - { - checkReadable(); - return delegate.getRoles(user); - } - - public synchronized Set getDatabasePrivileges(String user, String databaseName) - { - checkReadable(); - return delegate.getDatabasePrivileges(user, databaseName); - } - - public synchronized Set getTablePrivileges(String user, String databaseName, String tableName) + public synchronized Set listTablePrivileges(String databaseName, String tableName, PrestoPrincipal principal) { checkReadable(); SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); Action tableAction = tableActions.get(schemaTableName); if (tableAction == null) { - return delegate.getTablePrivileges(user, databaseName, tableName); + return delegate.listTablePrivileges(databaseName, tableName, principal); } switch (tableAction.getType()) { case ADD: case ALTER: { - if (!user.equals(tableAction.getData().getTable().getOwner())) { - throw new PrestoException(NOT_SUPPORTED, "Cannot access a table newly created in the transaction with a different user"); + if (principal.getType() == PrincipalType.ROLE) { + return ImmutableSet.of(); + } + if (!principal.getName().equals(tableAction.getData().getTable().getOwner())) { + return ImmutableSet.of(); } - Collection privileges = tableAction.getData().getPrincipalPrivileges().getUserPrivileges().get(user); + Collection privileges = tableAction.getData().getPrincipalPrivileges().getUserPrivileges().get(principal.getName()); return ImmutableSet.builder() .addAll(privileges) .add(new HivePrivilegeInfo(OWNERSHIP, true)) .build(); } case INSERT_EXISTING: - return delegate.getTablePrivileges(user, databaseName, tableName); + return delegate.listTablePrivileges(databaseName, tableName, principal); case DROP: throw new TableNotFoundException(schemaTableName); default: diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/UserTableKey.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/UserTableKey.java index 8f2ceaf3a645..990c20aefadd 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/UserTableKey.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/UserTableKey.java @@ -15,6 +15,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.prestosql.spi.security.PrestoPrincipal; import javax.annotation.concurrent.Immutable; @@ -26,22 +27,22 @@ @Immutable public class UserTableKey { - private final String user; + private final PrestoPrincipal principal; private final String database; private final String table; @JsonCreator - public UserTableKey(@JsonProperty("user") String user, @JsonProperty("table") String table, @JsonProperty("database") String database) + public UserTableKey(@JsonProperty("principal") PrestoPrincipal principal, @JsonProperty("database") String database, @JsonProperty("table") String table) { - this.user = requireNonNull(user, "user is null"); - this.table = requireNonNull(table, "table is null"); + this.principal = requireNonNull(principal, "user is null"); this.database = requireNonNull(database, "database is null"); + this.table = requireNonNull(table, "table is null"); } @JsonProperty - public String getUser() + public PrestoPrincipal getPrincipal() { - return user; + return principal; } @JsonProperty @@ -71,7 +72,7 @@ public boolean equals(Object o) return false; } UserTableKey that = (UserTableKey) o; - return Objects.equals(user, that.user) && + return Objects.equals(principal, that.principal) && Objects.equals(table, that.table) && Objects.equals(database, that.database); } @@ -79,14 +80,14 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(user, table, database); + return Objects.hash(principal, table, database); } @Override public String toString() { return toStringHelper(this) - .add("user", user) + .add("principal", principal) .add("table", table) .add("database", database) .toString(); diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/file/FileHiveMetastore.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/file/FileHiveMetastore.java index fc3faefd61db..c41e6628e417 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/file/FileHiveMetastore.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/file/FileHiveMetastore.java @@ -87,7 +87,6 @@ import static io.prestosql.plugin.hive.HiveErrorCode.HIVE_PARTITION_DROPPED_DURING_QUERY; import static io.prestosql.plugin.hive.HivePartitionManager.extractPartitionValues; import static io.prestosql.plugin.hive.HiveUtil.toPartitionValues; -import static io.prestosql.plugin.hive.metastore.Database.DEFAULT_DATABASE_NAME; import static io.prestosql.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege.OWNERSHIP; import static io.prestosql.plugin.hive.metastore.MetastoreUtil.makePartName; import static io.prestosql.plugin.hive.metastore.MetastoreUtil.verifyCanDropColumn; @@ -97,7 +96,6 @@ import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; import static io.prestosql.spi.security.PrincipalType.ROLE; import static io.prestosql.spi.security.PrincipalType.USER; -import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toList; import static java.util.stream.Collectors.toSet; @@ -942,53 +940,18 @@ public synchronized Map> getPartitionsByNames(String } @Override - public synchronized Set getRoles(String user) - { - ImmutableSet.Builder result = ImmutableSet.builder(); - result.add(PUBLIC_ROLE_NAME); - if (ADMIN_USERS.contains(user.toLowerCase(ENGLISH))) { - result.add(ADMIN_ROLE_NAME); - } - return result.build(); - } - - @Override - public synchronized Set getDatabasePrivileges(String user, String databaseName) - { - Set privileges = new HashSet<>(); - if (isDatabaseOwner(user, databaseName)) { - privileges.add(new HivePrivilegeInfo(OWNERSHIP, true)); - } - return privileges; - } - - @Override - public synchronized Set getTablePrivileges(String user, String databaseName, String tableName) + public synchronized Set listTablePrivileges(String databaseName, String tableName, PrestoPrincipal principal) { + ImmutableSet.Builder result = ImmutableSet.builder(); Table table = getRequiredTable(databaseName, tableName); - - Set privileges = new HashSet<>(); - if (user.equals(table.getOwner())) { - privileges.add(new HivePrivilegeInfo(OWNERSHIP, true)); - } - - Path permissionsDirectory = getPermissionsDirectory(table); - privileges.addAll(getTablePrivileges(permissionsDirectory, user, USER)); - for (String role : getRoles(user)) { - privileges.addAll(getTablePrivileges(permissionsDirectory, role, ROLE)); + if (principal.getType() == USER && table.getOwner().equals(principal.getName())) { + result.add(new HivePrivilegeInfo(OWNERSHIP, true)); } - return privileges; - } - - private synchronized Collection getTablePrivileges( - Path permissionsDirectory, - String principalName, - PrincipalType principalType) - { - Path permissionFilePath = getPermissionsPath(permissionsDirectory, principalName, principalType); - return readFile("permissions", permissionFilePath, permissionsCodec).orElse(ImmutableList.of()).stream() + Path permissionFilePath = getPermissionsPath(getPermissionsDirectory(table), principal.getName(), principal.getType()); + result.addAll(readFile("permissions", permissionFilePath, permissionsCodec).orElse(ImmutableList.of()).stream() .map(PermissionMetadata::toHivePrivilegeInfo) - .collect(toList()); + .collect(toSet())); + return result.build(); } @Override @@ -1000,7 +963,7 @@ public synchronized void grantTablePrivileges(String databaseName, String tableN @Override public synchronized void revokeTablePrivileges(String databaseName, String tableName, String grantee, Set privileges) { - Set currentPrivileges = getTablePrivileges(grantee, databaseName, tableName); + Set currentPrivileges = listTablePrivileges(databaseName, tableName, new PrestoPrincipal(USER, grantee)); currentPrivileges.removeAll(privileges); setTablePrivileges(grantee, USER, databaseName, tableName, currentPrivileges); @@ -1051,30 +1014,6 @@ private synchronized void deleteTablePrivileges(Table table) } } - private boolean isDatabaseOwner(String user, String databaseName) - { - // all users are "owners" of the default database - if (DEFAULT_DATABASE_NAME.equalsIgnoreCase(databaseName)) { - return true; - } - - Optional databaseMetadata = getDatabase(databaseName); - if (!databaseMetadata.isPresent()) { - return false; - } - - Database database = databaseMetadata.get(); - - // a database can be owned by a user or role - if (database.getOwnerType() == USER && user.equals(database.getOwnerName())) { - return true; - } - if (database.getOwnerType() == ROLE && getRoles(user).contains(database.getOwnerName())) { - return true; - } - return false; - } - private Path getDatabaseMetadataDirectory(String databaseName) { return new Path(catalogDirectory, databaseName); diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/glue/GlueHiveMetastore.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/glue/GlueHiveMetastore.java index 2516576e03fa..f60a605af9af 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/glue/GlueHiveMetastore.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/glue/GlueHiveMetastore.java @@ -825,27 +825,6 @@ public Set listRoleGrants(PrestoPrincipal principal) return ImmutableSet.of(); } - @Override - public Set getRoles(String user) - { - // all users belong to public role implicitly - return ImmutableSet.builder() - .add(PUBLIC_ROLE_NAME) - .build(); - } - - @Override - public Set getDatabasePrivileges(String user, String databaseName) - { - throw new PrestoException(NOT_SUPPORTED, "getDatabasePrivileges is not supported by Glue"); - } - - @Override - public Set getTablePrivileges(String user, String databaseName, String tableName) - { - throw new PrestoException(NOT_SUPPORTED, "getTablePrivileges is not supported by Glue"); - } - @Override public void grantTablePrivileges(String databaseName, String tableName, String grantee, Set privileges) { @@ -857,4 +836,10 @@ public void revokeTablePrivileges(String databaseName, String tableName, String { throw new PrestoException(NOT_SUPPORTED, "revokeTablePrivileges is not supported by Glue"); } + + @Override + public Set listTablePrivileges(String databaseName, String tableName, PrestoPrincipal principal) + { + throw new PrestoException(NOT_SUPPORTED, "listTablePrivileges is not supported by Glue"); + } } diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/BridgingHiveMetastore.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/BridgingHiveMetastore.java index d1fa1615eacb..6f672a6e32e8 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/BridgingHiveMetastore.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/BridgingHiveMetastore.java @@ -33,7 +33,6 @@ import io.prestosql.spi.statistics.ColumnStatisticType; import io.prestosql.spi.type.Type; import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; import javax.inject.Inject; @@ -48,7 +47,6 @@ import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.fromMetastoreApiTable; import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.isAvroTableWithSchemaSet; import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.toMetastoreApiDatabase; -import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.toMetastoreApiPrivilegeGrantInfo; import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.toMetastoreApiTable; import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; import static java.util.Objects.requireNonNull; @@ -330,38 +328,20 @@ public Set listRoleGrants(PrestoPrincipal principal) } @Override - public Set getRoles(String user) - { - return delegate.getRoles(user); - } - - @Override - public Set getDatabasePrivileges(String user, String databaseName) - { - return delegate.getDatabasePrivileges(user, databaseName); - } - - @Override - public Set getTablePrivileges(String user, String databaseName, String tableName) + public void grantTablePrivileges(String databaseName, String tableName, String grantee, Set privileges) { - return delegate.getTablePrivileges(user, databaseName, tableName); + delegate.grantTablePrivileges(databaseName, tableName, grantee, privileges); } @Override - public void grantTablePrivileges(String databaseName, String tableName, String grantee, Set privileges) + public void revokeTablePrivileges(String databaseName, String tableName, String grantee, Set privileges) { - Set privilegeGrantInfos = privileges.stream() - .map(privilege -> toMetastoreApiPrivilegeGrantInfo(grantee, privilege)) - .collect(Collectors.toSet()); - delegate.grantTablePrivileges(databaseName, tableName, grantee, privilegeGrantInfos); + delegate.revokeTablePrivileges(databaseName, tableName, grantee, privileges); } @Override - public void revokeTablePrivileges(String databaseName, String tableName, String grantee, Set privileges) + public Set listTablePrivileges(String databaseName, String tableName, PrestoPrincipal principal) { - Set privilegeGrantInfos = privileges.stream() - .map(privilege -> toMetastoreApiPrivilegeGrantInfo(grantee, privilege)) - .collect(Collectors.toSet()); - delegate.revokeTablePrivileges(databaseName, tableName, grantee, privilegeGrantInfos); + return delegate.listTablePrivileges(databaseName, tableName, principal); } } diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/HiveMetastore.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/HiveMetastore.java index 99b472bf0c0d..6d8fc6088f0f 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/HiveMetastore.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/HiveMetastore.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; import org.apache.hadoop.hive.metastore.api.Table; import java.util.List; @@ -36,9 +35,6 @@ import java.util.function.Function; import static io.prestosql.plugin.hive.HiveErrorCode.HIVE_INVALID_METADATA; -import static io.prestosql.plugin.hive.metastore.Database.DEFAULT_DATABASE_NAME; -import static org.apache.hadoop.hive.metastore.api.PrincipalType.ROLE; -import static org.apache.hadoop.hive.metastore.api.PrincipalType.USER; public interface HiveMetastore { @@ -100,39 +96,11 @@ public interface HiveMetastore Set listRoleGrants(PrestoPrincipal principal); - Set getRoles(String user); + void grantTablePrivileges(String databaseName, String tableName, String grantee, Set privileges); - Set getDatabasePrivileges(String user, String databaseName); + void revokeTablePrivileges(String databaseName, String tableName, String grantee, Set privileges); - Set getTablePrivileges(String user, String databaseName, String tableName); - - void grantTablePrivileges(String databaseName, String tableName, String grantee, Set privilegeGrantInfoSet); - - void revokeTablePrivileges(String databaseName, String tableName, String grantee, Set privilegeGrantInfoSet); - - default boolean isDatabaseOwner(String user, String databaseName) - { - // all users are "owners" of the default database - if (DEFAULT_DATABASE_NAME.equalsIgnoreCase(databaseName)) { - return true; - } - - Optional databaseMetadata = getDatabase(databaseName); - if (!databaseMetadata.isPresent()) { - return false; - } - - Database database = databaseMetadata.get(); - - // a database can be owned by a user or role - if (database.getOwnerType() == USER && user.equals(database.getOwnerName())) { - return true; - } - if (database.getOwnerType() == ROLE && getRoles(user).contains(database.getOwnerName())) { - return true; - } - return false; - } + Set listTablePrivileges(String databaseName, String tableName, PrestoPrincipal principal); default boolean isTableOwner(String user, String databaseName, String tableName) { diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/HiveMetastoreClient.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/HiveMetastoreClient.java index 9ed9b93900c4..2d36601f0d98 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/HiveMetastoreClient.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/HiveMetastoreClient.java @@ -19,7 +19,6 @@ import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege; import org.apache.hadoop.hive.metastore.api.HiveObjectRef; import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.hadoop.hive.metastore.api.PrivilegeBag; import org.apache.hadoop.hive.metastore.api.Role; @@ -115,9 +114,6 @@ List getPartitionsByNames(String databaseName, String tableName, List List listRoles(String principalName, PrincipalType principalType) throws TException; - PrincipalPrivilegeSet getPrivilegeSet(HiveObjectRef hiveObject, String userName, List groupNames) - throws TException; - List listPrivileges(String principalName, PrincipalType principalType, HiveObjectRef hiveObjectRef) throws TException; diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftHiveMetastore.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftHiveMetastore.java index 4d572d0f1a80..8d26ac0075e7 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftHiveMetastore.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftHiveMetastore.java @@ -51,11 +51,9 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.hadoop.hive.metastore.api.PrivilegeBag; import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; -import org.apache.hadoop.hive.metastore.api.Role; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.UnknownDBException; import org.apache.hadoop.hive.metastore.api.UnknownTableException; @@ -75,6 +73,7 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.function.Function; +import java.util.stream.Collectors; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Throwables.throwIfUnchecked; @@ -94,20 +93,18 @@ import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.fromPrestoPrincipalType; import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.fromRolePrincipalGrants; import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.getHiveBasicStatistics; -import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.toGrants; import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.toMetastoreApiPartition; +import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.toMetastoreApiPrivilegeGrantInfo; import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.updateStatisticsParameters; import static io.prestosql.spi.StandardErrorCode.ALREADY_EXISTS; import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.prestosql.spi.security.PrincipalType.USER; import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static java.util.function.Function.identity; import static java.util.stream.Collectors.toSet; import static org.apache.hadoop.hive.common.FileUtils.makePartName; -import static org.apache.hadoop.hive.metastore.api.HiveObjectType.DATABASE; import static org.apache.hadoop.hive.metastore.api.HiveObjectType.TABLE; -import static org.apache.hadoop.hive.metastore.api.PrincipalType.ROLE; -import static org.apache.hadoop.hive.metastore.api.PrincipalType.USER; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS; @ThreadSafe @@ -1157,60 +1154,11 @@ public List getPartitionsByNames(String databaseName, String tableNam } @Override - public Set getRoles(String user) - { - try { - return retry() - .stopOnIllegalExceptions() - .run("listRoles", stats.getLoadRoles().wrap(() -> { - try (HiveMetastoreClient client = clientProvider.createMetastoreClient()) { - List roles = client.listRoles(user, USER); - if (roles == null) { - return ImmutableSet.of(); - } - return ImmutableSet.copyOf(roles.stream() - .map(Role::getRoleName) - .collect(toSet())); - } - })); - } - catch (TException e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - catch (Exception e) { - throw propagate(e); - } - } - - @Override - public Set getDatabasePrivileges(String user, String databaseName) - { - ImmutableSet.Builder privileges = ImmutableSet.builder(); - - if (isDatabaseOwner(user, databaseName)) { - privileges.add(new HivePrivilegeInfo(OWNERSHIP, true)); - } - privileges.addAll(getUserPrivileges(new HivePrincipal(user, USER), new HiveObjectRef(DATABASE, databaseName, null, null, null))); - - return privileges.build(); - } - - @Override - public Set getTablePrivileges(String user, String databaseName, String tableName) - { - ImmutableSet.Builder privileges = ImmutableSet.builder(); - - if (isTableOwner(user, databaseName, tableName)) { - privileges.add(new HivePrivilegeInfo(OWNERSHIP, true)); - } - privileges.addAll(getUserPrivileges(new HivePrincipal(user, USER), new HiveObjectRef(TABLE, databaseName, tableName, null, null))); - - return privileges.build(); - } - - @Override - public void grantTablePrivileges(String databaseName, String tableName, String grantee, Set requestedPrivileges) + public void grantTablePrivileges(String databaseName, String tableName, String grantee, Set privileges) { + Set requestedPrivileges = privileges.stream() + .map(privilege -> toMetastoreApiPrivilegeGrantInfo(grantee, privilege)) + .collect(Collectors.toSet()); checkArgument(!containsAllPrivilege(requestedPrivileges), "\"ALL\" not supported in PrivilegeGrantInfo.privilege"); try { @@ -1219,7 +1167,7 @@ public void grantTablePrivileges(String databaseName, String tableName, String g .run("grantTablePrivileges", stats.getGrantTablePrivileges().wrap(() -> { try (HiveMetastoreClient metastoreClient = clientProvider.createMetastoreClient()) { HivePrincipal hivePrincipal = toHivePrincipal(grantee); - Set existingPrivileges = getTablePrivileges(hivePrincipal, databaseName, tableName); + Set existingPrivileges = listTablePrivileges(databaseName, tableName, new PrestoPrincipal(USER, grantee)); Set privilegesToGrant = new HashSet<>(requestedPrivileges); Iterator iterator = privilegesToGrant.iterator(); @@ -1258,8 +1206,11 @@ else if (existingPrivilege.isContainedIn(requestedPrivilege)) { } @Override - public void revokeTablePrivileges(String databaseName, String tableName, String grantee, Set requestedPrivileges) + public void revokeTablePrivileges(String databaseName, String tableName, String grantee, Set privileges) { + Set requestedPrivileges = privileges.stream() + .map(privilege -> toMetastoreApiPrivilegeGrantInfo(grantee, privilege)) + .collect(Collectors.toSet()); checkArgument(!containsAllPrivilege(requestedPrivileges), "\"ALL\" not supported in PrivilegeGrantInfo.privilege"); try { @@ -1268,7 +1219,7 @@ public void revokeTablePrivileges(String databaseName, String tableName, String .run("revokeTablePrivileges", stats.getRevokeTablePrivileges().wrap(() -> { try (HiveMetastoreClient metastoreClient = clientProvider.createMetastoreClient()) { HivePrincipal hivePrincipal = toHivePrincipal(grantee); - Set existingHivePrivileges = getTablePrivileges(hivePrincipal, databaseName, tableName).stream() + Set existingHivePrivileges = listTablePrivileges(databaseName, tableName, new PrestoPrincipal(USER, grantee)).stream() .map(HivePrivilegeInfo::getHivePrivilege) .collect(toSet()); @@ -1293,50 +1244,23 @@ public void revokeTablePrivileges(String databaseName, String tableName, String } } - private static PrivilegeBag buildPrivilegeBag( - String databaseName, - String tableName, - HivePrincipal hivePrincipal, - Set privilegeGrantInfos) - { - ImmutableList.Builder privilegeBagBuilder = ImmutableList.builder(); - for (PrivilegeGrantInfo privilegeGrantInfo : privilegeGrantInfos) { - privilegeBagBuilder.add( - new HiveObjectPrivilege( - new HiveObjectRef(TABLE, databaseName, tableName, null, null), - hivePrincipal.getPrincipalName(), - hivePrincipal.getPrincipalType(), - privilegeGrantInfo)); - } - return new PrivilegeBag(privilegeBagBuilder.build()); - } - - private static boolean containsAllPrivilege(Set requestedPrivileges) - { - return requestedPrivileges.stream() - .anyMatch(privilege -> privilege.getPrivilege().equalsIgnoreCase("all")); - } - - private Set getTablePrivileges(HivePrincipal hivePrincipal, String databaseName, String tableName) - { - if (hivePrincipal.getPrincipalType() == ROLE) { - return getRolePrivileges(hivePrincipal, new HiveObjectRef(TABLE, databaseName, tableName, null, null)); - } - return getUserPrivileges(hivePrincipal, new HiveObjectRef(TABLE, databaseName, tableName, null, null)); - } - - private Set getRolePrivileges(HivePrincipal hivePrincipal, HiveObjectRef hiveObjectRef) + @Override + public Set listTablePrivileges(String databaseName, String tableName, PrestoPrincipal principal) { - checkArgument(hivePrincipal.getPrincipalType() == ROLE, "Expected ROLE PrincipalType but found USER"); - try { return retry() .stopOnIllegalExceptions() .run("getListPrivileges", stats.getListPrivileges().wrap(() -> { try (HiveMetastoreClient client = clientProvider.createMetastoreClient()) { + Table table = client.getTable(databaseName, tableName); ImmutableSet.Builder privileges = ImmutableSet.builder(); - - List hiveObjectPrivilegeList = client.listPrivileges(hivePrincipal.getPrincipalName(), hivePrincipal.getPrincipalType(), hiveObjectRef); + if (principal.getType() == USER && table.getOwner().equals(principal.getName())) { + privileges.add(new HivePrivilegeInfo(OWNERSHIP, true)); + } + List hiveObjectPrivilegeList = client.listPrivileges( + principal.getName(), + fromPrestoPrincipalType(principal.getType()), + new HiveObjectRef(TABLE, databaseName, tableName, null, null)); for (HiveObjectPrivilege hiveObjectPrivilege : hiveObjectPrivilegeList) { privileges.addAll(parsePrivilege(hiveObjectPrivilege.getGrantInfo())); } @@ -1352,43 +1276,28 @@ private Set getRolePrivileges(HivePrincipal hivePrincipal, Hi } } - private Set getUserPrivileges(HivePrincipal hivePrincipal, HiveObjectRef objectReference) + private PrivilegeBag buildPrivilegeBag( + String databaseName, + String tableName, + HivePrincipal hivePrincipal, + Set privilegeGrantInfos) { - checkArgument(hivePrincipal.getPrincipalType() == USER, "Expected USER PrincipalType but found ROLE"); - - try { - return retry() - .stopOnIllegalExceptions() - .run("getPrivilegeSet", stats.getGetPrivilegeSet().wrap(() -> { - try (HiveMetastoreClient client = clientProvider.createMetastoreClient()) { - ImmutableSet.Builder privileges = ImmutableSet.builder(); - - String principalName = hivePrincipal.getPrincipalName(); - PrincipalPrivilegeSet privilegeSet = client.getPrivilegeSet(objectReference, principalName, null); - if (privilegeSet != null) { - Map> userPrivileges = privilegeSet.getUserPrivileges(); - if (userPrivileges != null) { - privileges.addAll(toGrants(userPrivileges.get(principalName))); - } - Map> rolePrivilegesMap = privilegeSet.getRolePrivileges(); - if (rolePrivilegesMap != null) { - for (List rolePrivileges : rolePrivilegesMap.values()) { - privileges.addAll(toGrants(rolePrivileges)); - } - } - // We do not add the group permissions as Hive does not seem to process these - } - - return privileges.build(); - } - })); - } - catch (TException e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - catch (Exception e) { - throw propagate(e); + ImmutableList.Builder privilegeBagBuilder = ImmutableList.builder(); + for (PrivilegeGrantInfo privilegeGrantInfo : privilegeGrantInfos) { + privilegeBagBuilder.add( + new HiveObjectPrivilege( + new HiveObjectRef(TABLE, databaseName, tableName, null, null), + hivePrincipal.getPrincipalName(), + hivePrincipal.getPrincipalType(), + privilegeGrantInfo)); } + return new PrivilegeBag(privilegeBagBuilder.build()); + } + + private boolean containsAllPrivilege(Set requestedPrivileges) + { + return requestedPrivileges.stream() + .anyMatch(privilege -> privilege.getPrivilege().equalsIgnoreCase("all")); } private RetryDriver retry() diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftHiveMetastoreClient.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftHiveMetastoreClient.java index 0759cdb0ac41..a2f0012bb85c 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftHiveMetastoreClient.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftHiveMetastoreClient.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.PartitionsStatsRequest; -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.hadoop.hive.metastore.api.PrivilegeBag; import org.apache.hadoop.hive.metastore.api.Role; @@ -261,13 +260,6 @@ public List listRoles(String principalName, PrincipalType principalType) return client.list_roles(principalName, principalType); } - @Override - public PrincipalPrivilegeSet getPrivilegeSet(HiveObjectRef hiveObject, String userName, List groupNames) - throws TException - { - return client.get_privilege_set(hiveObject, userName, groupNames); - } - @Override public List listPrivileges(String principalName, PrincipalType principalType, HiveObjectRef hiveObjectRef) throws TException diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftMetastoreUtil.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftMetastoreUtil.java index 853959a4c3a3..b3acd05008eb 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftMetastoreUtil.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftMetastoreUtil.java @@ -26,6 +26,7 @@ import io.prestosql.plugin.hive.metastore.Partition; import io.prestosql.plugin.hive.metastore.PartitionWithStatistics; import io.prestosql.plugin.hive.metastore.PrincipalPrivileges; +import io.prestosql.plugin.hive.metastore.SemiTransactionalHiveMetastore; import io.prestosql.plugin.hive.metastore.Storage; import io.prestosql.plugin.hive.metastore.StorageFormat; import io.prestosql.plugin.hive.metastore.Table; @@ -65,14 +66,19 @@ import java.math.BigInteger; 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; import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.OptionalDouble; import java.util.OptionalLong; +import java.util.Queue; import java.util.Set; +import java.util.function.Function; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.emptyToNull; @@ -87,7 +93,7 @@ import static io.prestosql.plugin.hive.metastore.HiveColumnStatistics.createDoubleColumnStatistics; import static io.prestosql.plugin.hive.metastore.HiveColumnStatistics.createIntegerColumnStatistics; import static io.prestosql.plugin.hive.metastore.HiveColumnStatistics.createStringColumnStatistics; -import static io.prestosql.plugin.hive.metastore.HivePrivilegeInfo.parsePrivilege; +import static io.prestosql.spi.security.PrincipalType.ROLE; import static io.prestosql.spi.statistics.ColumnStatisticType.MAX_VALUE; import static io.prestosql.spi.statistics.ColumnStatisticType.MAX_VALUE_SIZE_IN_BYTES; import static io.prestosql.spi.statistics.ColumnStatisticType.MIN_VALUE; @@ -110,6 +116,7 @@ import static java.lang.Math.round; import static java.lang.String.format; 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.booleanStats; import static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData.dateStats; @@ -197,6 +204,45 @@ public static org.apache.hadoop.hive.metastore.api.PrincipalType toMetastoreApiP } } + public static Set listApplicableRoles(PrestoPrincipal principal, Function> listRoleGrants) + { + Set result = new HashSet<>(); + Queue queue = new ArrayDeque<>(); + queue.add(principal); + while (!queue.isEmpty()) { + PrestoPrincipal current = queue.poll(); + Set grants = listRoleGrants.apply(current); + for (RoleGrant grant : grants) { + if (!result.contains(grant)) { + result.add(grant); + queue.add(new PrestoPrincipal(ROLE, grant.getRoleName())); + } + } + } + return ImmutableSet.copyOf(result); + } + + public static Set listApplicableRoles(SemiTransactionalHiveMetastore metastore, PrestoPrincipal principal) + { + return listApplicableRoles(principal, metastore::listRoleGrants) + .stream() + .map(RoleGrant::getRoleName) + .collect(toSet()); + } + + public static Set listApplicableTablePrivileges(SemiTransactionalHiveMetastore metastore, String databaseName, String tableName, PrestoPrincipal principal) + { + Set applicableRoles = listApplicableRoles(metastore, principal); + List principals = new ArrayList<>(); + principals.add(principal); + applicableRoles.stream().map(role -> new PrestoPrincipal(ROLE, role)).forEach(principals::add); + ImmutableSet.Builder result = ImmutableSet.builder(); + for (PrestoPrincipal current : principals) { + result.addAll(metastore.listTablePrivileges(databaseName, tableName, current)); + } + return result.build(); + } + public static org.apache.hadoop.hive.metastore.api.Partition toMetastoreApiPartition(PartitionWithStatistics partitionWithStatistics) { org.apache.hadoop.hive.metastore.api.Partition partition = toMetastoreApiPartition(partitionWithStatistics.getPartition()); @@ -215,23 +261,10 @@ public static org.apache.hadoop.hive.metastore.api.Partition toMetastoreApiParti return result; } - public static Set toGrants(List userGrants) - { - if (userGrants == null) { - return ImmutableSet.of(); - } - - ImmutableSet.Builder privileges = ImmutableSet.builder(); - for (PrivilegeGrantInfo userGrant : userGrants) { - privileges.addAll(parsePrivilege(userGrant)); - } - return privileges.build(); - } - public static Database fromMetastoreApiDatabase(org.apache.hadoop.hive.metastore.api.Database database) { String ownerName = "PUBLIC"; - PrincipalType ownerType = PrincipalType.ROLE; + PrincipalType ownerType = ROLE; if (database.getOwnerName() != null) { ownerName = database.getOwnerName(); ownerType = fromMetastoreApiPrincipalType(database.getOwnerType()); @@ -498,7 +531,7 @@ public static PrincipalType fromMetastoreApiPrincipalType(org.apache.hadoop.hive case USER: return PrincipalType.USER; case ROLE: - return PrincipalType.ROLE; + return ROLE; default: throw new IllegalArgumentException("Unsupported principal type: " + principalType); } diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/security/SqlStandardAccessControl.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/security/SqlStandardAccessControl.java index 482933e4473d..9d7d9ecb81ce 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/security/SqlStandardAccessControl.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/security/SqlStandardAccessControl.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableSet; import io.prestosql.plugin.hive.HiveConnectorId; import io.prestosql.plugin.hive.HiveTransactionHandle; +import io.prestosql.plugin.hive.metastore.Database; import io.prestosql.plugin.hive.metastore.HivePrivilegeInfo; import io.prestosql.plugin.hive.metastore.SemiTransactionalHiveMetastore; import io.prestosql.spi.connector.ConnectorAccessControl; @@ -28,17 +29,20 @@ import javax.inject.Inject; +import java.util.HashSet; import java.util.Optional; import java.util.Set; import java.util.function.Function; -import java.util.stream.Collectors; +import static io.prestosql.plugin.hive.metastore.Database.DEFAULT_DATABASE_NAME; import static io.prestosql.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege; import static io.prestosql.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege.DELETE; import static io.prestosql.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege.INSERT; import static io.prestosql.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege.OWNERSHIP; import static io.prestosql.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege.SELECT; 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.spi.security.AccessDeniedException.denyAddColumn; import static io.prestosql.spi.security.AccessDeniedException.denyCreateRole; import static io.prestosql.spi.security.AccessDeniedException.denyCreateSchema; @@ -59,7 +63,10 @@ import static io.prestosql.spi.security.AccessDeniedException.denyRevokeTablePrivilege; import static io.prestosql.spi.security.AccessDeniedException.denySelectTable; import static io.prestosql.spi.security.AccessDeniedException.denySetCatalogSessionProperty; +import static io.prestosql.spi.security.PrincipalType.ROLE; +import static io.prestosql.spi.security.PrincipalType.USER; import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toSet; public class SqlStandardAccessControl implements ConnectorAccessControl @@ -296,9 +303,9 @@ public void checkCanRevokeRoles(ConnectorTransactionHandle transactionHandle, Id private boolean checkDatabasePermission(ConnectorTransactionHandle transaction, Identity identity, String schemaName, HivePrivilege... requiredPrivileges) { SemiTransactionalHiveMetastore metastore = metastoreProvider.apply(((HiveTransactionHandle) transaction)); - Set privilegeSet = metastore.getDatabasePrivileges(identity.getUser(), schemaName).stream() + Set privilegeSet = getDatabasePrivileges(metastore, identity.getUser(), schemaName).stream() .map(HivePrivilegeInfo::getHivePrivilege) - .collect(Collectors.toSet()); + .collect(toSet()); return privilegeSet.containsAll(ImmutableSet.copyOf(requiredPrivileges)); } @@ -311,7 +318,11 @@ private boolean isDatabaseOwner(ConnectorTransactionHandle transaction, Identity private boolean getGrantOptionForPrivilege(ConnectorTransactionHandle transaction, Identity identity, Privilege privilege, SchemaTableName tableName) { SemiTransactionalHiveMetastore metastore = metastoreProvider.apply(((HiveTransactionHandle) transaction)); - return metastore.getTablePrivileges(identity.getUser(), tableName.getSchemaName(), tableName.getTableName()) + return listApplicableTablePrivileges( + metastore, + tableName.getSchemaName(), + tableName.getTableName(), + new PrestoPrincipal(USER, identity.getUser())) .contains(new HivePrivilegeInfo(toHivePrivilege(privilege), true)); } @@ -326,15 +337,53 @@ private boolean checkTablePermission(ConnectorTransactionHandle transaction, Ide } SemiTransactionalHiveMetastore metastore = metastoreProvider.apply(((HiveTransactionHandle) transaction)); - Set privilegeSet = metastore.getTablePrivileges(identity.getUser(), tableName.getSchemaName(), tableName.getTableName()).stream() + Set privilegeSet = listApplicableTablePrivileges( + metastore, + tableName.getSchemaName(), + tableName.getTableName(), + new PrestoPrincipal(USER, identity.getUser())) + .stream() .map(HivePrivilegeInfo::getHivePrivilege) - .collect(Collectors.toSet()); + .collect(toSet()); return privilegeSet.containsAll(ImmutableSet.copyOf(requiredPrivileges)); } private boolean isAdmin(ConnectorTransactionHandle transaction, Identity identity) { SemiTransactionalHiveMetastore metastore = metastoreProvider.apply(((HiveTransactionHandle) transaction)); - return metastore.getRoles(identity.getUser()).contains(ADMIN_ROLE_NAME); + return listApplicableRoles(metastore, new PrestoPrincipal(USER, identity.getUser())).contains(ADMIN_ROLE_NAME); + } + + private static Set getDatabasePrivileges(SemiTransactionalHiveMetastore metastore, String user, String databaseName) + { + Set privileges = new HashSet<>(); + if (isDatabaseOwner(metastore, user, databaseName)) { + privileges.add(new HivePrivilegeInfo(OWNERSHIP, true)); + } + return privileges; + } + + private static boolean isDatabaseOwner(SemiTransactionalHiveMetastore metastore, String user, String databaseName) + { + // all users are "owners" of the default database + if (DEFAULT_DATABASE_NAME.equalsIgnoreCase(databaseName)) { + return true; + } + + Optional databaseMetadata = metastore.getDatabase(databaseName); + if (!databaseMetadata.isPresent()) { + return false; + } + + Database database = databaseMetadata.get(); + + // a database can be owned by a user or role + if (database.getOwnerType() == USER && user.equals(database.getOwnerName())) { + return true; + } + if (database.getOwnerType() == ROLE && listApplicableRoles(metastore, new PrestoPrincipal(USER, user)).contains(database.getOwnerName())) { + return true; + } + return false; } } diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/TestRecordingHiveMetastore.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/TestRecordingHiveMetastore.java index 30aea4078773..cf4496ce6dc0 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/TestRecordingHiveMetastore.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/TestRecordingHiveMetastore.java @@ -135,9 +135,7 @@ private void validateMetadata(ExtendedHiveMetastore hiveMetastore) assertEquals(hiveMetastore.getPartitionNames("database", "table"), Optional.of(ImmutableList.of("value"))); assertEquals(hiveMetastore.getPartitionNamesByParts("database", "table", ImmutableList.of("value")), Optional.of(ImmutableList.of("value"))); assertEquals(hiveMetastore.getPartitionsByNames("database", "table", ImmutableList.of("value")), ImmutableMap.of("value", Optional.of(PARTITION))); - assertEquals(hiveMetastore.getRoles("user"), ImmutableSet.of("role1", "role2")); - assertEquals(hiveMetastore.getDatabasePrivileges("user", "database"), ImmutableSet.of(PRIVILEGE_INFO)); - assertEquals(hiveMetastore.getTablePrivileges("user", "database", "table"), ImmutableSet.of(PRIVILEGE_INFO)); + assertEquals(hiveMetastore.listTablePrivileges("database", "table", new PrestoPrincipal(USER, "user")), ImmutableSet.of(PRIVILEGE_INFO)); assertEquals(hiveMetastore.listRoles(), ImmutableSet.of("role")); assertEquals(hiveMetastore.listRoleGrants(new PrestoPrincipal(USER, "user")), ImmutableSet.of(ROLE_GRANT)); } @@ -264,25 +262,9 @@ public Map> getPartitionsByNames(String databaseName } @Override - public Set getRoles(String user) + public Set listTablePrivileges(String database, String table, PrestoPrincipal prestoPrincipal) { - return ImmutableSet.of("role1", "role2"); - } - - @Override - public Set getDatabasePrivileges(String user, String databaseName) - { - if (user.equals("user") && databaseName.equals("database")) { - return ImmutableSet.of(PRIVILEGE_INFO); - } - - return ImmutableSet.of(); - } - - @Override - public Set getTablePrivileges(String user, String databaseName, String tableName) - { - if (user.equals("user") && databaseName.equals("database") && tableName.equals("table")) { + if (database.equals("database") && table.equals("table") && prestoPrincipal.getType() == USER && prestoPrincipal.getName().equals("user")) { return ImmutableSet.of(PRIVILEGE_INFO); } diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/UnimplementedHiveMetastore.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/UnimplementedHiveMetastore.java index 06f1a210d9e2..679451b48aa5 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/UnimplementedHiveMetastore.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/UnimplementedHiveMetastore.java @@ -192,19 +192,7 @@ public void alterPartition(String databaseName, String tableName, PartitionWithS } @Override - public Set getRoles(String user) - { - throw new UnsupportedOperationException(); - } - - @Override - public Set getDatabasePrivileges(String user, String databaseName) - { - throw new UnsupportedOperationException(); - } - - @Override - public Set getTablePrivileges(String user, String databaseName, String tableName) + public Set listTablePrivileges(String databaseName, String tableName, PrestoPrincipal prestoPrincipal) { throw new UnsupportedOperationException(); } diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/InMemoryHiveMetastore.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/InMemoryHiveMetastore.java index 2408623cb778..6ecdea1558f3 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/InMemoryHiveMetastore.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/InMemoryHiveMetastore.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; import org.apache.hadoop.hive.metastore.api.PrincipalType; -import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; import org.apache.hadoop.hive.metastore.api.Table; import javax.annotation.concurrent.GuardedBy; @@ -44,13 +43,10 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.net.URI; -import java.util.Collection; import java.util.EnumSet; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -58,12 +54,10 @@ import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.prestosql.plugin.hive.HiveBasicStatistics.createEmptyStatistics; import static io.prestosql.plugin.hive.HiveUtil.toPartitionValues; -import static io.prestosql.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege.OWNERSHIP; import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.toMetastoreApiPartition; import static io.prestosql.spi.StandardErrorCode.SCHEMA_NOT_EMPTY; import static java.util.Locale.US; @@ -72,14 +66,10 @@ import static org.apache.hadoop.hive.metastore.TableType.EXTERNAL_TABLE; import static org.apache.hadoop.hive.metastore.TableType.MANAGED_TABLE; import static org.apache.hadoop.hive.metastore.TableType.VIRTUAL_VIEW; -import static org.apache.hadoop.hive.metastore.api.PrincipalType.ROLE; -import static org.apache.hadoop.hive.metastore.api.PrincipalType.USER; public class InMemoryHiveMetastore implements HiveMetastore { - private static final String PUBLIC_ROLE_NAME = "public"; - @GuardedBy("this") private final Map databases = new HashMap<>(); @GuardedBy("this") @@ -93,8 +83,6 @@ public class InMemoryHiveMetastore @GuardedBy("this") private final Map partitionColumnStatistics = new HashMap<>(); @GuardedBy("this") - private final Map> roleGrants = new HashMap<>(); - @GuardedBy("this") private final Map> tablePrivileges = new HashMap<>(); private final File baseDirectory; @@ -206,22 +194,7 @@ public synchronized void createTable(Table table) PrincipalPrivilegeSet privileges = table.getPrivileges(); if (privileges != null) { - for (Entry> entry : privileges.getUserPrivileges().entrySet()) { - String user = entry.getKey(); - Set userPrivileges = entry.getValue().stream() - .map(HivePrivilegeInfo::parsePrivilege) - .flatMap(Collection::stream) - .collect(toImmutableSet()); - setTablePrivileges(user, USER, table.getDbName(), table.getTableName(), userPrivileges); - } - for (Entry> entry : privileges.getRolePrivileges().entrySet()) { - String role = entry.getKey(); - Set rolePrivileges = entry.getValue().stream() - .map(HivePrivilegeInfo::parsePrivilege) - .flatMap(Collection::stream) - .collect(toImmutableSet()); - setTablePrivileges(role, ROLE, table.getDbName(), table.getTableName(), rolePrivileges); - } + throw new UnsupportedOperationException(); } } @@ -513,76 +486,21 @@ public Set listRoleGrants(PrestoPrincipal principal) } @Override - public synchronized Set getRoles(String user) - { - return roleGrants.getOrDefault(user, ImmutableSet.of(PUBLIC_ROLE_NAME)); - } - - public synchronized void setUserRoles(String user, Set roles) - { - if (!roles.contains(PUBLIC_ROLE_NAME)) { - roles = ImmutableSet.builder() - .addAll(roles) - .add(PUBLIC_ROLE_NAME) - .build(); - } - roleGrants.put(user, ImmutableSet.copyOf(roles)); - } - - @Override - public synchronized Set getDatabasePrivileges(String user, String databaseName) + public Set listTablePrivileges(String databaseName, String tableName, PrestoPrincipal principal) { - Set privileges = new HashSet<>(); - if (isDatabaseOwner(user, databaseName)) { - privileges.add(new HivePrivilegeInfo(OWNERSHIP, true)); - } - return privileges; - } - - @Override - public synchronized Set getTablePrivileges(String user, String databaseName, String tableName) - { - Set privileges = new HashSet<>(); - if (isTableOwner(user, databaseName, tableName)) { - privileges.add(new HivePrivilegeInfo(OWNERSHIP, true)); - } - privileges.addAll(tablePrivileges.getOrDefault(new PrincipalTableKey(user, USER, tableName, databaseName), ImmutableSet.of())); - for (String role : getRoles(user)) { - privileges.addAll(tablePrivileges.getOrDefault(new PrincipalTableKey(role, ROLE, tableName, databaseName), ImmutableSet.of())); - } - return privileges; - } - - public synchronized void setTablePrivileges(String principalName, - PrincipalType principalType, - String databaseName, - String tableName, - Set privileges) - { - tablePrivileges.put(new PrincipalTableKey(principalName, principalType, tableName, databaseName), ImmutableSet.copyOf(privileges)); + throw new UnsupportedOperationException(); } @Override - public synchronized void grantTablePrivileges(String databaseName, String tableName, String grantee, Set privilegeGrantInfoSet) + public void grantTablePrivileges(String databaseName, String tableName, String grantee, Set privileges) { - Set hivePrivileges = privilegeGrantInfoSet.stream() - .map(HivePrivilegeInfo::parsePrivilege) - .flatMap(Collection::stream) - .collect(toImmutableSet()); - - setTablePrivileges(grantee, USER, databaseName, tableName, hivePrivileges); + throw new UnsupportedOperationException(); } @Override - public synchronized void revokeTablePrivileges(String databaseName, String tableName, String grantee, Set privilegeGrantInfoSet) + public void revokeTablePrivileges(String databaseName, String tableName, String grantee, Set privileges) { - Set currentPrivileges = getTablePrivileges(grantee, databaseName, tableName); - currentPrivileges.removeAll(privilegeGrantInfoSet.stream() - .map(HivePrivilegeInfo::parsePrivilege) - .flatMap(Collection::stream) - .collect(toImmutableSet())); - - setTablePrivileges(grantee, USER, databaseName, tableName, currentPrivileges); + throw new UnsupportedOperationException(); } private static boolean isParentDir(File directory, File baseDirectory) diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/MockHiveMetastoreClient.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/MockHiveMetastoreClient.java index 00d8d196f8bf..17032750bc29 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/MockHiveMetastoreClient.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/MockHiveMetastoreClient.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.hadoop.hive.metastore.api.PrivilegeBag; import org.apache.hadoop.hive.metastore.api.Role; @@ -309,12 +308,6 @@ public List listRoles(String principalName, PrincipalType principalType) throw new UnsupportedOperationException(); } - @Override - public PrincipalPrivilegeSet getPrivilegeSet(HiveObjectRef hiveObject, String userName, List groupNames) - { - throw new UnsupportedOperationException(); - } - @Override public List listPrivileges(String principalName, PrincipalType principalType, HiveObjectRef hiveObjectRef) {