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 87cc2c2c1879..9ce7af995d22 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 @@ -837,12 +837,13 @@ private static Table buildTableObject( private static PrincipalPrivileges buildInitialPrivilegeSet(String tableOwner) { + PrestoPrincipal grantor = new PrestoPrincipal(USER, tableOwner); return new PrincipalPrivileges( ImmutableMultimap.builder() - .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.SELECT, true)) - .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.INSERT, true)) - .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.UPDATE, true)) - .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.DELETE, true)) + .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.SELECT, true, grantor)) + .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.INSERT, true, grantor)) + .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.UPDATE, true, grantor)) + .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.DELETE, true, grantor)) .build(), ImmutableMultimap.of()); } @@ -1820,7 +1821,7 @@ public void grantTablePrivileges(ConnectorSession session, SchemaTableName schem String tableName = schemaTableName.getTableName(); Set hivePrivilegeInfos = privileges.stream() - .map(privilege -> new HivePrivilegeInfo(toHivePrivilege(privilege), grantOption)) + .map(privilege -> new HivePrivilegeInfo(toHivePrivilege(privilege), grantOption, new PrestoPrincipal(USER, session.getUser()))) .collect(toSet()); metastore.grantTablePrivileges(schemaName, tableName, grantee, hivePrivilegeInfos); @@ -1833,7 +1834,7 @@ public void revokeTablePrivileges(ConnectorSession session, SchemaTableName sche String tableName = schemaTableName.getTableName(); Set hivePrivilegeInfos = privileges.stream() - .map(privilege -> new HivePrivilegeInfo(toHivePrivilege(privilege), grantOption)) + .map(privilege -> new HivePrivilegeInfo(toHivePrivilege(privilege), grantOption, new PrestoPrincipal(USER, session.getUser()))) .collect(toSet()); metastore.revokeTablePrivileges(schemaName, tableName, grantee, hivePrivilegeInfos); diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/HivePrivilegeInfo.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/HivePrivilegeInfo.java index e8dbf746ef88..ebf6504bd1d4 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/HivePrivilegeInfo.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/HivePrivilegeInfo.java @@ -16,6 +16,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableSet; +import io.prestosql.spi.security.PrestoPrincipal; import io.prestosql.spi.security.Privilege; import io.prestosql.spi.security.PrivilegeInfo; @@ -43,14 +44,17 @@ public enum HivePrivilege private final HivePrivilege hivePrivilege; private final boolean grantOption; + private final PrestoPrincipal grantor; @JsonCreator public HivePrivilegeInfo( @JsonProperty("hivePrivilege") HivePrivilege hivePrivilege, - @JsonProperty("grantOption") boolean grantOption) + @JsonProperty("grantOption") boolean grantOption, + @JsonProperty("grantor") PrestoPrincipal grantor) { this.hivePrivilege = requireNonNull(hivePrivilege, "hivePrivilege is null"); this.grantOption = grantOption; + this.grantor = requireNonNull(grantor, "grantor is null"); } @JsonProperty @@ -65,6 +69,12 @@ public boolean isGrantOption() return grantOption; } + @JsonProperty + public PrestoPrincipal getGrantor() + { + return grantor; + } + public static HivePrivilege toHivePrivilege(Privilege privilege) { switch (privilege) { 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 a162adb4f167..2661f1b6e7c3 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 @@ -82,6 +82,7 @@ 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.USER; import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static org.apache.hadoop.hive.common.FileUtils.makePartName; @@ -788,7 +789,7 @@ public synchronized Set listTablePrivileges(String databaseNa Collection privileges = tableAction.getData().getPrincipalPrivileges().getUserPrivileges().get(principal.getName()); return ImmutableSet.builder() .addAll(privileges) - .add(new HivePrivilegeInfo(OWNERSHIP, true)) + .add(new HivePrivilegeInfo(OWNERSHIP, true, new PrestoPrincipal(USER, principal.getName()))) .build(); } case INSERT_EXISTING: 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 a6a47ac7db2e..4a21daac7c74 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 @@ -944,7 +944,7 @@ public synchronized Set listTablePrivileges(String databaseNa ImmutableSet.Builder result = ImmutableSet.builder(); Table table = getRequiredTable(databaseName, tableName); if (principal.getType() == USER && table.getOwner().equals(principal.getName())) { - result.add(new HivePrivilegeInfo(OWNERSHIP, true)); + result.add(new HivePrivilegeInfo(OWNERSHIP, true, principal)); } Path permissionFilePath = getPermissionsPath(getPermissionsDirectory(table), principal); result.addAll(readFile("permissions", permissionFilePath, permissionsCodec).orElse(ImmutableList.of()).stream() diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/file/PermissionMetadata.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/file/PermissionMetadata.java index e49699182f9f..82cefe26cead 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/file/PermissionMetadata.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/file/PermissionMetadata.java @@ -17,7 +17,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import io.prestosql.plugin.hive.metastore.HivePrivilegeInfo; import io.prestosql.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege; +import io.prestosql.spi.security.PrestoPrincipal; +import static io.prestosql.spi.security.PrincipalType.USER; import static java.util.Objects.requireNonNull; public class PermissionMetadata @@ -54,6 +56,6 @@ public boolean isGrantOption() public HivePrivilegeInfo toHivePrivilegeInfo() { - return new HivePrivilegeInfo(permission, grantOption); + return new HivePrivilegeInfo(permission, grantOption, new PrestoPrincipal(USER, "admin")); } } 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 04cc393053a4..bc0e5849f40b 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 @@ -92,7 +92,6 @@ import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.getHiveBasicStatistics; import static io.prestosql.plugin.hive.metastore.thrift.ThriftMetastoreUtil.parsePrivilege; 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; @@ -1155,7 +1154,7 @@ public List getPartitionsByNames(String databaseName, String tableNam public void grantTablePrivileges(String databaseName, String tableName, PrestoPrincipal grantee, Set privileges) { Set requestedPrivileges = privileges.stream() - .map(privilege -> toMetastoreApiPrivilegeGrantInfo(grantee, privilege)) + .map(ThriftMetastoreUtil::toMetastoreApiPrivilegeGrantInfo) .collect(Collectors.toSet()); checkArgument(!containsAllPrivilege(requestedPrivileges), "\"ALL\" not supported in PrivilegeGrantInfo.privilege"); @@ -1206,7 +1205,7 @@ else if (existingPrivilege.isContainedIn(requestedPrivilege)) { public void revokeTablePrivileges(String databaseName, String tableName, PrestoPrincipal grantee, Set privileges) { Set requestedPrivileges = privileges.stream() - .map(privilege -> toMetastoreApiPrivilegeGrantInfo(grantee, privilege)) + .map(ThriftMetastoreUtil::toMetastoreApiPrivilegeGrantInfo) .collect(Collectors.toSet()); checkArgument(!containsAllPrivilege(requestedPrivileges), "\"ALL\" not supported in PrivilegeGrantInfo.privilege"); @@ -1251,7 +1250,7 @@ public Set listTablePrivileges(String databaseName, String ta Table table = client.getTable(databaseName, tableName); ImmutableSet.Builder privileges = ImmutableSet.builder(); if (principal.getType() == USER && table.getOwner().equals(principal.getName())) { - privileges.add(new HivePrivilegeInfo(OWNERSHIP, true)); + privileges.add(new HivePrivilegeInfo(OWNERSHIP, true, principal)); } List hiveObjectPrivilegeList = client.listPrivileges( principal.getName(), 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 f610af9e574d..fb8a8f19d60b 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 @@ -125,6 +125,7 @@ import static java.lang.Math.round; import static java.lang.String.format; 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; import static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData.binaryStats; @@ -168,38 +169,38 @@ public static org.apache.hadoop.hive.metastore.api.Table toMetastoreApiTable(Tab result.setParameters(table.getParameters()); result.setPartitionKeys(table.getPartitionColumns().stream().map(ThriftMetastoreUtil::toMetastoreApiFieldSchema).collect(toList())); result.setSd(makeStorageDescriptor(table.getTableName(), table.getDataColumns(), table.getStorage())); - result.setPrivileges(toMetastoreApiPrincipalPrivilegeSet(new PrestoPrincipal(USER, table.getOwner()), privileges)); + result.setPrivileges(toMetastoreApiPrincipalPrivilegeSet(privileges)); result.setViewOriginalText(table.getViewOriginalText().orElse(null)); result.setViewExpandedText(table.getViewExpandedText().orElse(null)); return result; } - private static PrincipalPrivilegeSet toMetastoreApiPrincipalPrivilegeSet(PrestoPrincipal grantee, PrincipalPrivileges privileges) + private static PrincipalPrivilegeSet toMetastoreApiPrincipalPrivilegeSet(PrincipalPrivileges privileges) { ImmutableMap.Builder> userPrivileges = ImmutableMap.builder(); for (Map.Entry> entry : privileges.getUserPrivileges().asMap().entrySet()) { userPrivileges.put(entry.getKey(), entry.getValue().stream() - .map(privilegeInfo -> toMetastoreApiPrivilegeGrantInfo(grantee, privilegeInfo)) + .map(ThriftMetastoreUtil::toMetastoreApiPrivilegeGrantInfo) .collect(toList())); } ImmutableMap.Builder> rolePrivileges = ImmutableMap.builder(); for (Map.Entry> entry : privileges.getRolePrivileges().asMap().entrySet()) { rolePrivileges.put(entry.getKey(), entry.getValue().stream() - .map(privilegeInfo -> toMetastoreApiPrivilegeGrantInfo(grantee, privilegeInfo)) + .map(ThriftMetastoreUtil::toMetastoreApiPrivilegeGrantInfo) .collect(toList())); } return new PrincipalPrivilegeSet(userPrivileges.build(), ImmutableMap.of(), rolePrivileges.build()); } - public static PrivilegeGrantInfo toMetastoreApiPrivilegeGrantInfo(PrestoPrincipal grantee, HivePrivilegeInfo privilegeInfo) + public static PrivilegeGrantInfo toMetastoreApiPrivilegeGrantInfo(HivePrivilegeInfo privilegeInfo) { return new PrivilegeGrantInfo( privilegeInfo.getHivePrivilege().name().toLowerCase(Locale.ENGLISH), 0, - grantee.getName(), - fromPrestoPrincipalType(grantee.getType()), + privilegeInfo.getGrantor().getName(), + fromPrestoPrincipalType(privilegeInfo.getGrantor().getType()), privilegeInfo.isGrantOption()); } @@ -585,6 +586,7 @@ public static org.apache.hadoop.hive.metastore.api.PrincipalType fromPrestoPrinc public static PrincipalType fromMetastoreApiPrincipalType(org.apache.hadoop.hive.metastore.api.PrincipalType principalType) { + requireNonNull(principalType, "principalType is null"); switch (principalType) { case USER: return USER; @@ -657,21 +659,22 @@ public static Set parsePrivilege(PrivilegeGrantInfo userGrant { boolean withGrantOption = userGrant.isGrantOption(); String name = userGrant.getPrivilege().toUpperCase(ENGLISH); + PrestoPrincipal grantor = new PrestoPrincipal(ThriftMetastoreUtil.fromMetastoreApiPrincipalType(userGrant.getGrantorType()), userGrant.getGrantor()); switch (name) { case "ALL": return Arrays.stream(HivePrivilegeInfo.HivePrivilege.values()) - .map(hivePrivilege -> new HivePrivilegeInfo(hivePrivilege, withGrantOption)) + .map(hivePrivilege -> new HivePrivilegeInfo(hivePrivilege, withGrantOption, grantor)) .collect(toImmutableSet()); case "SELECT": - return ImmutableSet.of(new HivePrivilegeInfo(SELECT, withGrantOption)); + return ImmutableSet.of(new HivePrivilegeInfo(SELECT, withGrantOption, grantor)); case "INSERT": - return ImmutableSet.of(new HivePrivilegeInfo(INSERT, withGrantOption)); + return ImmutableSet.of(new HivePrivilegeInfo(INSERT, withGrantOption, grantor)); case "UPDATE": - return ImmutableSet.of(new HivePrivilegeInfo(UPDATE, withGrantOption)); + return ImmutableSet.of(new HivePrivilegeInfo(UPDATE, withGrantOption, grantor)); case "DELETE": - return ImmutableSet.of(new HivePrivilegeInfo(DELETE, withGrantOption)); + return ImmutableSet.of(new HivePrivilegeInfo(DELETE, withGrantOption, grantor)); case "OWNERSHIP": - return ImmutableSet.of(new HivePrivilegeInfo(OWNERSHIP, withGrantOption)); + return ImmutableSet.of(new HivePrivilegeInfo(OWNERSHIP, withGrantOption, grantor)); default: throw new IllegalArgumentException("Unsupported privilege name: " + name); } 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 2b7378f53d67..9232756fc8b1 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,7 +16,6 @@ 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; import io.prestosql.spi.connector.ConnectorTransactionHandle; @@ -419,7 +418,8 @@ private boolean hasGrantOptionForPrivilege(ConnectorTransactionHandle transactio tableName.getSchemaName(), tableName.getTableName(), identity.getUser()) - .contains(new HivePrivilegeInfo(toHivePrivilege(privilege), true)); + .stream() + .anyMatch(privilegeInfo -> privilegeInfo.getHivePrivilege().equals(toHivePrivilege(privilege)) && privilegeInfo.isGrantOption()); } private boolean hasAdminOptionForRoles(ConnectorTransactionHandle transaction, ConnectorIdentity identity, Set roles) diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHiveClient.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHiveClient.java index 36761e26039c..1bd6ac7c6903 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHiveClient.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHiveClient.java @@ -85,6 +85,7 @@ import io.prestosql.spi.predicate.Range; import io.prestosql.spi.predicate.TupleDomain; import io.prestosql.spi.predicate.ValueSet; +import io.prestosql.spi.security.PrestoPrincipal; import io.prestosql.spi.statistics.ColumnStatistics; import io.prestosql.spi.statistics.TableStatistics; import io.prestosql.spi.type.ArrayType; @@ -221,6 +222,7 @@ import static io.prestosql.spi.StandardErrorCode.TRANSACTION_CONFLICT; import static io.prestosql.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; import static io.prestosql.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; +import static io.prestosql.spi.security.PrincipalType.USER; import static io.prestosql.spi.type.BigintType.BIGINT; import static io.prestosql.spi.type.BooleanType.BOOLEAN; import static io.prestosql.spi.type.CharType.createCharType; @@ -1074,7 +1076,7 @@ protected void doTestMismatchSchemaTable( // alter the table schema try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); - PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(session.getUser()); + PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(session); Table oldTable = transaction.getMetastore(schemaName).getTable(schemaName, tableName).get(); HiveTypeTranslator hiveTypeTranslator = new HiveTypeTranslator(); List dataColumns = tableAfter.stream() @@ -2180,7 +2182,7 @@ public void testTableCreationIgnoreExisting() ConnectorSession session = newSession(); String schemaName = schemaTableName.getSchemaName(); String tableName = schemaTableName.getTableName(); - PrincipalPrivileges privileges = testingPrincipalPrivilege(session.getUser()); + PrincipalPrivileges privileges = testingPrincipalPrivilege(session); Path targetPath; try { try (Transaction transaction = newTransaction()) { @@ -4289,7 +4291,7 @@ private void createEmptyTable(SchemaTableName schemaTableName, HiveStorageFormat .setBucketProperty(bucketProperty) .setSerdeParameters(ImmutableMap.of()); - PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(tableOwner); + PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(tableOwner, session.getUser()); transaction.getMetastore(schemaName).createTable(session, tableBuilder.build(), principalPrivileges, Optional.empty(), true, EMPTY_TABLE_STATISTICS); transaction.commit(); @@ -4312,7 +4314,7 @@ private void alterBucketProperty(SchemaTableName schemaTableName, Optional table = transaction.getMetastore(schemaName).getTable(schemaName, tableName); Table.Builder tableBuilder = Table.builder(table.get()); tableBuilder.getStorageBuilder().setBucketProperty(bucketProperty); - PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(tableOwner); + PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(tableOwner, session.getUser()); // hack: replaceView can be used as replaceTable despite its name transaction.getMetastore(schemaName).replaceView(schemaName, tableName, tableBuilder.build(), principalPrivileges); @@ -4320,14 +4322,19 @@ private void alterBucketProperty(SchemaTableName schemaTableName, Optionalbuilder() - .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.SELECT, true)) - .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.INSERT, true)) - .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.UPDATE, true)) - .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.DELETE, true)) + .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.SELECT, true, new PrestoPrincipal(USER, grantor))) + .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.INSERT, true, new PrestoPrincipal(USER, grantor))) + .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.UPDATE, true, new PrestoPrincipal(USER, grantor))) + .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.DELETE, true, new PrestoPrincipal(USER, grantor))) .build(), ImmutableMultimap.of()); } 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 cf4496ce6dc0..ff0cf7adf28b 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 @@ -25,6 +25,7 @@ import io.prestosql.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege; import io.prestosql.plugin.hive.metastore.SortingColumn.Order; 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; @@ -97,7 +98,7 @@ public class TestRecordingHiveMetastore OptionalLong.of(1235), OptionalLong.of(1), OptionalLong.of(8)))); - private static final HivePrivilegeInfo PRIVILEGE_INFO = new HivePrivilegeInfo(HivePrivilege.SELECT, true); + private static final HivePrivilegeInfo PRIVILEGE_INFO = new HivePrivilegeInfo(HivePrivilege.SELECT, true, new PrestoPrincipal(PrincipalType.USER, "grantor")); private static final RoleGrant ROLE_GRANT = new RoleGrant(new PrestoPrincipal(USER, "grantee"), "role", true); @Test