diff --git a/.travis.yml b/.travis.yml index 24ab0a7450990..79be948b35105 100644 --- a/.travis.yml +++ b/.travis.yml @@ -30,6 +30,7 @@ matrix: - env: PRODUCT_TESTS_SUITE=suite-2 - env: PRODUCT_TESTS_SUITE=suite-3 - env: PRODUCT_TESTS_SUITE=suite-4 + - env: PRODUCT_TESTS_SUITE=suite-5 - env: HIVE_TESTS=true - env: KUDU_TESTS=true @@ -44,6 +45,8 @@ matrix: # - if: type IN (push, cron) # env: PRODUCT_TESTS_SUITE=suite-4 PRODUCT_TESTS_CONFIG=config-hdp3 # - if: type IN (push, cron) +# env: PRODUCT_TESTS_SUITE=suite-5 PRODUCT_TESTS_CONFIG=config-hdp3 +# - if: type IN (push, cron) # env: HIVE_TESTS=true PRODUCT_TESTS_CONFIG=config-hdp3 dist: xenial diff --git a/presto-docs/src/main/sphinx/connector/hive-security.rst b/presto-docs/src/main/sphinx/connector/hive-security.rst index ae06c8d22772a..740caa89d6690 100644 --- a/presto-docs/src/main/sphinx/connector/hive-security.rst +++ b/presto-docs/src/main/sphinx/connector/hive-security.rst @@ -125,6 +125,8 @@ Property Name Description ================================================== ============================================================ ``hive.metastore.authentication.type`` Hive metastore authentication type. +``hive.metastore.thrift.impersonation.enabled`` Enable Hive metastore end user impersonation. + ``hive.metastore.service.principal`` The Kerberos principal of the Hive metastore service. ``hive.metastore.client.principal`` The Kerberos principal that Presto will use when connecting @@ -145,6 +147,13 @@ Thrift service using SASL and authenticate using Kerberos. This property is optional; the default is ``NONE``. +``hive.metastore.thrift.impersonation.enabled`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Enable end-user Hive metastore impersonation. + +This property is optional; the default is ``false``. + ``hive.metastore.service.principal`` ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -214,6 +223,7 @@ Example configuration with ``KERBEROS`` authentication .. code-block:: none hive.metastore.authentication.type=KERBEROS + hive.metastore.thrift.impersonation.enabled=true hive.metastore.service.principal=hive/hive-metastore-host.example.com@EXAMPLE.COM hive.metastore.client.principal=presto@EXAMPLE.COM hive.metastore.client.keytab=/etc/presto/hive.keytab @@ -394,12 +404,6 @@ Keytab files must be distributed to every node in the cluster that runs Presto. :ref:`Additional Information About Keytab Files.` -Impersonation Accessing the Hive Metastore ------------------------------------------- - -Presto does not currently support impersonating the end user when accessing the -Hive metastore. - .. _configuring-hadoop-impersonation: Impersonation in Hadoop diff --git a/presto-docs/src/main/sphinx/connector/hive.rst b/presto-docs/src/main/sphinx/connector/hive.rst index 62795b36d4a3f..889206770477d 100644 --- a/presto-docs/src/main/sphinx/connector/hive.rst +++ b/presto-docs/src/main/sphinx/connector/hive.rst @@ -211,6 +211,8 @@ Property Name Description Possible values are ``NONE`` or ``KERBEROS`` (defaults to ``NONE``). +``hive.metastore.thrift.impersonation.enabled`` Enable Hive metastore end user impersonation. + ``hive.metastore.service.principal`` The Kerberos principal of the Hive metastore service. ``hive.metastore.client.principal`` The Kerberos principal that Presto will use when connecting diff --git a/presto-geospatial/src/test/java/io/prestosql/plugin/geospatial/TestSpatialJoins.java b/presto-geospatial/src/test/java/io/prestosql/plugin/geospatial/TestSpatialJoins.java index fac50e26cd788..c1d424ff5e77f 100644 --- a/presto-geospatial/src/test/java/io/prestosql/plugin/geospatial/TestSpatialJoins.java +++ b/presto-geospatial/src/test/java/io/prestosql/plugin/geospatial/TestSpatialJoins.java @@ -21,6 +21,7 @@ import io.prestosql.plugin.hive.HiveConfig; import io.prestosql.plugin.hive.HiveHdfsConfiguration; import io.prestosql.plugin.hive.HivePlugin; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.authentication.NoHdfsAuthentication; import io.prestosql.plugin.hive.metastore.Database; import io.prestosql.plugin.hive.metastore.file.FileHiveMetastore; @@ -33,6 +34,7 @@ import java.util.Optional; import static io.prestosql.SystemSessionProperties.SPATIAL_PARTITIONING_TABLE_NAME; +import static io.prestosql.testing.TestingConnectorSession.SESSION; import static io.prestosql.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; @@ -82,11 +84,13 @@ private static DistributedQueryRunner createQueryRunner() HdfsEnvironment hdfsEnvironment = new HdfsEnvironment(hdfsConfiguration, hiveConfig, new NoHdfsAuthentication()); FileHiveMetastore metastore = new FileHiveMetastore(hdfsEnvironment, baseDir.toURI().toString(), "test"); - metastore.createDatabase(Database.builder() - .setDatabaseName("default") - .setOwnerName("public") - .setOwnerType(PrincipalType.ROLE) - .build()); + metastore.createDatabase( + new HiveIdentity(SESSION), + Database.builder() + .setDatabaseName("default") + .setOwnerName("public") + .setOwnerType(PrincipalType.ROLE) + .build()); queryRunner.installPlugin(new HivePlugin("hive", Optional.of(metastore))); queryRunner.createCatalog("hive", "hive"); diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/CreateEmptyPartitionProcedure.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/CreateEmptyPartitionProcedure.java index 89fee5222df59..34beaa817c261 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/CreateEmptyPartitionProcedure.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/CreateEmptyPartitionProcedure.java @@ -19,6 +19,7 @@ import io.airlift.slice.Slices; import io.prestosql.plugin.hive.LocationService.WriteInfo; import io.prestosql.plugin.hive.PartitionUpdate.UpdateMode; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.HiveMetastore; import io.prestosql.spi.PrestoException; import io.prestosql.spi.classloader.ThreadContextClassLoader; @@ -111,7 +112,7 @@ private void doCreateEmptyPartition(ConnectorSession session, String schema, Str .map(String.class::cast) .collect(toImmutableList()); - if (metastore.getPartition(schema, table, partitionStringValues).isPresent()) { + if (metastore.getPartition(new HiveIdentity(session), schema, table, partitionStringValues).isPresent()) { throw new PrestoException(ALREADY_EXISTS, "Partition already exists"); } String partitionName = FileUtils.makePartName(actualPartitionColumnNames, partitionStringValues); 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 035d0a3514396..81adea788bf49 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 @@ -29,6 +29,7 @@ import io.airlift.slice.Slice; import io.prestosql.plugin.hive.HdfsEnvironment.HdfsContext; import io.prestosql.plugin.hive.LocationService.WriteInfo; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Column; import io.prestosql.plugin.hive.metastore.Database; import io.prestosql.plugin.hive.metastore.HiveColumnStatistics; @@ -313,7 +314,7 @@ public List listSchemaNames(ConnectorSession session) public HiveTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) { requireNonNull(tableName, "tableName is null"); - Optional table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); + Optional
table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()); if (!table.isPresent()) { return null; } @@ -346,7 +347,7 @@ public ConnectorTableHandle getTableHandleForStatisticsCollection(ConnectorSessi return null; } Optional>> partitionValuesList = getPartitionList(analyzeProperties); - ConnectorTableMetadata tableMetadata = getTableMetadata(handle.getSchemaTableName()); + ConnectorTableMetadata tableMetadata = getTableMetadata(session, handle.getSchemaTableName()); handle = handle.withAnalyzePartitionValues(partitionValuesList); List partitionedBy = getPartitionedBy(tableMetadata.getProperties()); @@ -376,14 +377,14 @@ public Optional getSystemTable(ConnectorSession session, SchemaTabl return getPartitionsSystemTable(session, tableName, SystemTableHandler.PARTITIONS.getSourceTableName(tableName)); } if (SystemTableHandler.PROPERTIES.matches(tableName)) { - return getPropertiesSystemTable(tableName, SystemTableHandler.PROPERTIES.getSourceTableName(tableName)); + return getPropertiesSystemTable(session, tableName, SystemTableHandler.PROPERTIES.getSourceTableName(tableName)); } return Optional.empty(); } - private Optional getPropertiesSystemTable(SchemaTableName tableName, SchemaTableName sourceTableName) + private Optional getPropertiesSystemTable(ConnectorSession session, SchemaTableName tableName, SchemaTableName sourceTableName) { - Optional
table = metastore.getTable(sourceTableName.getSchemaName(), sourceTableName.getTableName()); + Optional
table = metastore.getTable(new HiveIdentity(session), sourceTableName.getSchemaName(), sourceTableName.getTableName()); if (!table.isPresent() || table.get().getTableType().equals(TableType.VIRTUAL_VIEW.name())) { throw new TableNotFoundException(tableName); } @@ -437,7 +438,7 @@ private Optional getPartitionsSystemTable(ConnectorSession session, Predicate> targetPredicate = convertToPredicate(targetTupleDomain); Constraint targetConstraint = new Constraint(targetTupleDomain, targetPredicate); Iterable> records = () -> - stream(partitionManager.getPartitions(metastore, sourceTableHandle, targetConstraint).getPartitions()) + stream(partitionManager.getPartitions(metastore, new HiveIdentity(session), sourceTableHandle, targetConstraint).getPartitions()) .map(hivePartition -> IntStream.range(0, partitionColumns.size()) .mapToObj(fieldIdToColumnHandle::get) @@ -452,13 +453,13 @@ private Optional getPartitionsSystemTable(ConnectorSession session, @Override public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle tableHandle) { - return getTableMetadata(((HiveTableHandle) tableHandle).getSchemaTableName()); + return getTableMetadata(session, ((HiveTableHandle) tableHandle).getSchemaTableName()); } - private ConnectorTableMetadata getTableMetadata(SchemaTableName tableName) + private ConnectorTableMetadata getTableMetadata(ConnectorSession session, SchemaTableName tableName) { try { - return doGetTableMetadata(tableName); + return doGetTableMetadata(session, tableName); } catch (PrestoException e) { throw e; @@ -470,9 +471,9 @@ private ConnectorTableMetadata getTableMetadata(SchemaTableName tableName) } } - private ConnectorTableMetadata doGetTableMetadata(SchemaTableName tableName) + private ConnectorTableMetadata doGetTableMetadata(ConnectorSession session, SchemaTableName tableName) { - Optional
table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); + Optional
table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()); if (!table.isPresent() || table.get().getTableType().equals(TableType.VIRTUAL_VIEW.name())) { throw new TableNotFoundException(tableName); } @@ -610,7 +611,7 @@ private List listSchemas(ConnectorSession session, Optional sche public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) { SchemaTableName tableName = ((HiveTableHandle) tableHandle).getSchemaTableName(); - Table table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()) + Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); return hiveColumnHandles(table).stream() .collect(toImmutableMap(HiveColumnHandle::getName, identity())); @@ -624,7 +625,7 @@ public Map> listTableColumns(ConnectorSess ImmutableMap.Builder> columns = ImmutableMap.builder(); for (SchemaTableName tableName : listTables(session, prefix)) { try { - columns.put(tableName, getTableMetadata(tableName).getColumns()); + columns.put(tableName, getTableMetadata(session, tableName).getColumns()); } catch (HiveViewNotSupportedException e) { // view is not supported @@ -648,7 +649,7 @@ public TableStatistics getTableStatistics(ConnectorSession session, ConnectorTab .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue)); Map columnTypes = columns.entrySet().stream() .collect(toImmutableMap(Map.Entry::getKey, entry -> getColumnMetadata(session, tableHandle, entry.getValue()).getType())); - HivePartitionResult partitionResult = partitionManager.getPartitions(metastore, tableHandle, constraint); + HivePartitionResult partitionResult = partitionManager.getPartitions(metastore, new HiveIdentity(session), tableHandle, constraint); List partitions = partitionManager.getPartitionsAsList(partitionResult); return hiveStatisticsProvider.getTableStatistics(session, ((HiveTableHandle) tableHandle).getSchemaTableName(), columns, columnTypes, partitions); } @@ -660,7 +661,7 @@ private List listTables(ConnectorSession session, SchemaTablePr } SchemaTableName tableName = prefix.toSchemaTableName(); try { - if (!metastore.getTable(tableName.getSchemaName(), tableName.getTableName()).isPresent()) { + if (!metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()).isPresent()) { return ImmutableList.of(); } } @@ -699,7 +700,7 @@ public void createSchema(ConnectorSession session, String schemaName, Map new TableNotFoundException(handle.getSchemaTableName())); if (table.getParameters().containsKey(AVRO_SCHEMA_URL_KEY) || table.getStorage().getSerdeParameters().containsKey(AVRO_SCHEMA_URL_KEY)) { throw new PrestoException(NOT_SUPPORTED, "ALTER TABLE not supported when Avro schema url is set"); @@ -1024,21 +1025,21 @@ private void failIfAvroSchemaIsSet(HiveTableHandle handle) public void renameTable(ConnectorSession session, ConnectorTableHandle tableHandle, SchemaTableName newTableName) { HiveTableHandle handle = (HiveTableHandle) tableHandle; - metastore.renameTable(handle.getSchemaName(), handle.getTableName(), newTableName.getSchemaName(), newTableName.getTableName()); + metastore.renameTable(new HiveIdentity(session), handle.getSchemaName(), handle.getTableName(), newTableName.getSchemaName(), newTableName.getTableName()); } @Override public void setTableComment(ConnectorSession session, ConnectorTableHandle tableHandle, Optional comment) { HiveTableHandle handle = (HiveTableHandle) tableHandle; - metastore.commentTable(handle.getSchemaName(), handle.getTableName(), comment); + metastore.commentTable(new HiveIdentity(session), handle.getSchemaName(), handle.getTableName(), comment); } @Override public void dropTable(ConnectorSession session, ConnectorTableHandle tableHandle) { HiveTableHandle handle = (HiveTableHandle) tableHandle; - Optional
target = metastore.getTable(handle.getSchemaName(), handle.getTableName()); + Optional
target = metastore.getTable(new HiveIdentity(session), handle.getSchemaName(), handle.getTableName()); if (!target.isPresent()) { throw new TableNotFoundException(handle.getSchemaTableName()); } @@ -1050,7 +1051,7 @@ public ConnectorTableHandle beginStatisticsCollection(ConnectorSession session, { verifyJvmTimeZone(); SchemaTableName tableName = ((HiveTableHandle) tableHandle).getSchemaTableName(); - metastore.getTable(tableName.getSchemaName(), tableName.getTableName()) + metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); return tableHandle; } @@ -1058,9 +1059,10 @@ public ConnectorTableHandle beginStatisticsCollection(ConnectorSession session, @Override public void finishStatisticsCollection(ConnectorSession session, ConnectorTableHandle tableHandle, Collection computedStatistics) { + HiveIdentity identity = new HiveIdentity(session); HiveTableHandle handle = (HiveTableHandle) tableHandle; SchemaTableName tableName = handle.getSchemaTableName(); - Table table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()) + Table table = metastore.getTable(identity, tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(handle.getSchemaTableName())); List partitionColumns = table.getPartitionColumns(); @@ -1076,7 +1078,7 @@ public void finishStatisticsCollection(ConnectorSession session, ConnectorTableH if (partitionColumns.isEmpty()) { // commit analyze to unpartitioned table - metastore.setTableStatistics(table, createPartitionStatistics(session, columnTypes, computedStatisticsMap.get(ImmutableList.of()))); + metastore.setTableStatistics(identity, table, createPartitionStatistics(session, columnTypes, computedStatisticsMap.get(ImmutableList.of()))); } else { List> partitionValuesList; @@ -1084,7 +1086,7 @@ public void finishStatisticsCollection(ConnectorSession session, ConnectorTableH partitionValuesList = handle.getAnalyzePartitionValues().get(); } else { - partitionValuesList = metastore.getPartitionNames(handle.getSchemaName(), handle.getTableName()) + partitionValuesList = metastore.getPartitionNames(identity, handle.getSchemaName(), handle.getTableName()) .orElseThrow(() -> new TableNotFoundException(((HiveTableHandle) tableHandle).getSchemaTableName())) .stream() .map(HiveUtil::toPartitionValues) @@ -1110,7 +1112,7 @@ public void finishStatisticsCollection(ConnectorSession session, ConnectorTableH } } verify(usedComputedStatistics == computedStatistics.size(), "All computed statistics must be used"); - metastore.setPartitionStatistics(table, partitionStatistics.build()); + metastore.setPartitionStatistics(identity, table, partitionStatistics.build()); } } @@ -1156,7 +1158,7 @@ public HiveOutputTableHandle beginCreateTable(ConnectorSession session, Connecto schemaName, tableName, columnHandles, - metastore.generatePageSinkMetadata(schemaTableName), + metastore.generatePageSinkMetadata(new HiveIdentity(session), schemaTableName), locationHandle, tableStorageFormat, partitionStorageFormat, @@ -1354,8 +1356,9 @@ public HiveInsertTableHandle beginInsert(ConnectorSession session, ConnectorTabl { verifyJvmTimeZone(); + HiveIdentity identity = new HiveIdentity(session); SchemaTableName tableName = ((HiveTableHandle) tableHandle).getSchemaTableName(); - Table table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()) + Table table = metastore.getTable(identity, tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); checkTableIsWritable(table, writesToNonManagedTablesEnabled); @@ -1384,7 +1387,7 @@ public HiveInsertTableHandle beginInsert(ConnectorSession session, ConnectorTabl tableName.getSchemaName(), tableName.getTableName(), handles, - metastore.generatePageSinkMetadata(tableName), + metastore.generatePageSinkMetadata(identity, tableName), locationHandle, table.getStorage().getBucketProperty(), tableStorageFormat, @@ -1408,7 +1411,7 @@ public Optional finishInsert(ConnectorSession session, HiveStorageFormat tableStorageFormat = handle.getTableStorageFormat(); partitionUpdates = PartitionUpdate.mergePartitionUpdates(partitionUpdates); - Table table = metastore.getTable(handle.getSchemaName(), handle.getTableName()) + Table table = metastore.getTable(new HiveIdentity(session), handle.getSchemaName(), handle.getTableName()) .orElseThrow(() -> new TableNotFoundException(handle.getSchemaTableName())); if (!table.getStorage().getStorageFormat().getInputFormat().equals(tableStorageFormat.getInputFormat()) && isRespectTableFormat(session)) { throw new PrestoException(HIVE_CONCURRENT_MODIFICATION_DETECTED, "Table format changed during insert"); @@ -1446,7 +1449,7 @@ public Optional finishInsert(ConnectorSession session, if (partitionUpdate.getUpdateMode() == OVERWRITE) { // get privileges from existing table - PrincipalPrivileges principalPrivileges = fromHivePrivilegeInfos(metastore.listTablePrivileges(handle.getSchemaName(), handle.getTableName(), null)); + PrincipalPrivileges principalPrivileges = fromHivePrivilegeInfos(metastore.listTablePrivileges(new HiveIdentity(session), handle.getSchemaName(), handle.getTableName(), null)); // first drop it metastore.dropTable(session, handle.getSchemaName(), handle.getTableName()); @@ -1574,6 +1577,7 @@ private static Map getColumnStatistics(Map properties = ImmutableMap.builder() .put(TABLE_COMMENT, "Presto View") .put(PRESTO_VIEW_FLAG, "true") @@ -1600,13 +1604,13 @@ public void createView(ConnectorSession session, SchemaTableName viewName, Conne Table table = tableBuilder.build(); PrincipalPrivileges principalPrivileges = buildInitialPrivilegeSet(session.getUser()); - Optional
existing = metastore.getTable(viewName.getSchemaName(), viewName.getTableName()); + Optional
existing = metastore.getTable(identity, viewName.getSchemaName(), viewName.getTableName()); if (existing.isPresent()) { if (!replace || !HiveUtil.isPrestoView(existing.get())) { throw new ViewAlreadyExistsException(viewName); } - metastore.replaceTable(viewName.getSchemaName(), viewName.getTableName(), table, principalPrivileges); + metastore.replaceTable(identity, viewName.getSchemaName(), viewName.getTableName(), table, principalPrivileges); return; } @@ -1648,7 +1652,7 @@ public List listViews(ConnectorSession session, Optional getView(ConnectorSession session, SchemaTableName viewName) { - return metastore.getTable(viewName.getSchemaName(), viewName.getTableName()) + return metastore.getTable(new HiveIdentity(session), viewName.getSchemaName(), viewName.getTableName()) .filter(HiveUtil::isPrestoView) .map(view -> { ConnectorViewDefinition definition = decodeViewData(view.getViewOriginalText() @@ -1690,7 +1694,7 @@ public OptionalLong executeDelete(ConnectorSession session, ConnectorTableHandle { HiveTableHandle handle = (HiveTableHandle) deleteHandle; - Optional
table = metastore.getTable(handle.getSchemaName(), handle.getTableName()); + Optional
table = metastore.getTable(new HiveIdentity(session), handle.getSchemaName(), handle.getTableName()); if (!table.isPresent()) { throw new TableNotFoundException(handle.getSchemaTableName()); } @@ -1699,7 +1703,7 @@ public OptionalLong executeDelete(ConnectorSession session, ConnectorTableHandle metastore.truncateUnpartitionedTable(session, handle.getSchemaName(), handle.getTableName()); } else { - for (HivePartition hivePartition : partitionManager.getOrLoadPartitions(metastore, handle)) { + for (HivePartition hivePartition : partitionManager.getOrLoadPartitions(metastore, new HiveIdentity(session), handle)) { metastore.dropPartition(session, handle.getSchemaName(), handle.getTableName(), toPartitionValues(hivePartition.getPartitionId())); } } @@ -1725,7 +1729,7 @@ public ConnectorTableProperties getTableProperties(ConnectorSession session, Con HiveTableHandle hiveTable = (HiveTableHandle) table; List partitionColumns = ImmutableList.copyOf(hiveTable.getPartitionColumns()); - List partitions = partitionManager.getOrLoadPartitions(metastore, hiveTable); + List partitions = partitionManager.getOrLoadPartitions(metastore, new HiveIdentity(session), hiveTable); TupleDomain predicate = createPredicate(partitionColumns, partitions); @@ -1766,7 +1770,7 @@ public Optional> applyFilter(C HiveTableHandle handle = (HiveTableHandle) tableHandle; checkArgument(!handle.getAnalyzePartitionValues().isPresent() || constraint.getSummary().isAll(), "Analyze should not have a constraint"); - HivePartitionResult partitionResult = partitionManager.getPartitions(metastore, handle, constraint); + HivePartitionResult partitionResult = partitionManager.getPartitions(metastore, new HiveIdentity(session), handle, constraint); HiveTableHandle newHandle = partitionManager.applyPartitionResult(handle, partitionResult); if (handle.getPartitions().equals(newHandle.getPartitions()) && @@ -1920,7 +1924,7 @@ public Optional getInsertLayout(ConnectorSession sessio { HiveTableHandle hiveTableHandle = (HiveTableHandle) tableHandle; SchemaTableName tableName = hiveTableHandle.getSchemaTableName(); - Table table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()) + Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); if (!isHiveBucketingV1(table) && table.getStorage().getBucketProperty().isPresent()) { diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/HivePageSinkProvider.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/HivePageSinkProvider.java index 9b29b650d5157..4235994b55b35 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/HivePageSinkProvider.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/HivePageSinkProvider.java @@ -20,6 +20,7 @@ import io.airlift.event.client.EventClient; import io.airlift.json.JsonCodec; import io.airlift.units.DataSize; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.HiveMetastore; import io.prestosql.plugin.hive.metastore.HivePageSinkMetadataProvider; import io.prestosql.plugin.hive.metastore.SortingColumn; @@ -146,7 +147,7 @@ private ConnectorPageSink createPageSink(HiveWritableTableHandle handle, boolean handle.getLocationHandle(), locationService, session.getQueryId(), - new HivePageSinkMetadataProvider(handle.getPageSinkMetadata(), memoizeMetastore(metastore, perTransactionMetastoreCacheMaximumSize)), + new HivePageSinkMetadataProvider(handle.getPageSinkMetadata(), memoizeMetastore(metastore, perTransactionMetastoreCacheMaximumSize), new HiveIdentity(session)), typeManager, hdfsEnvironment, pageSorter, diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/HivePartitionManager.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/HivePartitionManager.java index fc58360efb96b..40e3a3dc01e7b 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/HivePartitionManager.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/HivePartitionManager.java @@ -19,6 +19,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import io.airlift.slice.Slice; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.SemiTransactionalHiveMetastore; import io.prestosql.plugin.hive.metastore.Table; import io.prestosql.plugin.hive.util.HiveBucketing.HiveBucketFilter; @@ -116,7 +117,7 @@ public HivePartitionManager( this.typeManager = requireNonNull(typeManager, "typeManager is null"); } - public HivePartitionResult getPartitions(SemiTransactionalHiveMetastore metastore, ConnectorTableHandle tableHandle, Constraint constraint) + public HivePartitionResult getPartitions(SemiTransactionalHiveMetastore metastore, HiveIdentity identity, ConnectorTableHandle tableHandle, Constraint constraint) { HiveTableHandle hiveTableHandle = (HiveTableHandle) tableHandle; TupleDomain effectivePredicate = constraint.getSummary() @@ -130,7 +131,7 @@ public HivePartitionResult getPartitions(SemiTransactionalHiveMetastore metastor return new HivePartitionResult(partitionColumns, ImmutableList.of(), none(), none(), none(), hiveBucketHandle, Optional.empty()); } - Table table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()) + Table table = metastore.getTable(identity, tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); Optional bucketFilter = getHiveBucketFilter(table, effectivePredicate); @@ -159,7 +160,7 @@ public HivePartitionResult getPartitions(SemiTransactionalHiveMetastore metastor .collect(toImmutableList()); } else { - List partitionNames = getFilteredPartitionNames(metastore, tableName, partitionColumns, effectivePredicate); + List partitionNames = getFilteredPartitionNames(metastore, identity, tableName, partitionColumns, effectivePredicate); partitionsIterable = () -> partitionNames.stream() // Apply extra filters which could not be done by getFilteredPartitionNames .map(partitionName -> parseValuesAndFilterPartition(tableName, partitionName, partitionColumns, partitionTypes, effectivePredicate, predicate)) @@ -232,10 +233,10 @@ public HiveTableHandle applyPartitionResult(HiveTableHandle handle, HivePartitio handle.getAnalyzePartitionValues()); } - public List getOrLoadPartitions(SemiTransactionalHiveMetastore metastore, HiveTableHandle table) + public List getOrLoadPartitions(SemiTransactionalHiveMetastore metastore, HiveIdentity identity, HiveTableHandle table) { return table.getPartitions().orElseGet(() -> - getPartitionsAsList(getPartitions(metastore, table, new Constraint(table.getEnforcedConstraint())))); + getPartitionsAsList(getPartitions(metastore, identity, table, new Constraint(table.getEnforcedConstraint())))); } private static TupleDomain toCompactTupleDomain(TupleDomain effectivePredicate, int threshold) @@ -287,7 +288,7 @@ private boolean partitionMatches(List partitionColumns, TupleD return constraint.test(partition.getKeys()); } - private List getFilteredPartitionNames(SemiTransactionalHiveMetastore metastore, SchemaTableName tableName, List partitionKeys, TupleDomain effectivePredicate) + private List getFilteredPartitionNames(SemiTransactionalHiveMetastore metastore, HiveIdentity identity, SchemaTableName tableName, List partitionKeys, TupleDomain effectivePredicate) { checkArgument(effectivePredicate.getDomains().isPresent()); @@ -350,7 +351,7 @@ else if (type instanceof TinyintType } // fetch the partition names - return metastore.getPartitionNamesByParts(tableName.getSchemaName(), tableName.getTableName(), filter) + return metastore.getPartitionNamesByParts(identity, tableName.getSchemaName(), tableName.getTableName(), filter) .orElseThrow(() -> new TableNotFoundException(tableName)); } diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitManager.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitManager.java index 133f4b223ba8a..c5915d5d27f2c 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitManager.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitManager.java @@ -21,6 +21,7 @@ import io.airlift.concurrent.BoundedExecutor; import io.airlift.stats.CounterStat; import io.airlift.units.DataSize; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Column; import io.prestosql.plugin.hive.metastore.Partition; import io.prestosql.plugin.hive.metastore.SemiTransactionalHiveMetastore; @@ -175,7 +176,7 @@ public ConnectorSplitSource getSplits( // get table metadata SemiTransactionalHiveMetastore metastore = metastoreProvider.apply((HiveTransactionHandle) transaction); - Table table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()) + Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); // verify table is not marked as non-readable @@ -185,7 +186,7 @@ public ConnectorSplitSource getSplits( } // get partitions - List partitions = partitionManager.getOrLoadPartitions(metastore, hiveTable); + List partitions = partitionManager.getOrLoadPartitions(metastore, new HiveIdentity(session), hiveTable); // short circuit if we don't have any partitions if (partitions.isEmpty()) { @@ -204,7 +205,7 @@ public ConnectorSplitSource getSplits( // sort partitions partitions = Ordering.natural().onResultOf(HivePartition::getPartitionId).reverse().sortedCopy(partitions); - Iterable hivePartitions = getPartitionMetadata(metastore, table, tableName, partitions, bucketHandle.map(HiveBucketHandle::toTableBucketProperty)); + Iterable hivePartitions = getPartitionMetadata(session, metastore, table, tableName, partitions, bucketHandle.map(HiveBucketHandle::toTableBucketProperty)); HiveSplitLoader hiveSplitLoader = new BackgroundHiveSplitLoader( table, @@ -262,7 +263,7 @@ public CounterStat getHighMemorySplitSource() return highMemorySplitSourceCounter; } - private Iterable getPartitionMetadata(SemiTransactionalHiveMetastore metastore, Table table, SchemaTableName tableName, List hivePartitions, Optional bucketProperty) + private Iterable getPartitionMetadata(ConnectorSession session, SemiTransactionalHiveMetastore metastore, Table table, SchemaTableName tableName, List hivePartitions, Optional bucketProperty) { if (hivePartitions.isEmpty()) { return ImmutableList.of(); @@ -278,6 +279,7 @@ private Iterable getPartitionMetadata(SemiTransactionalHi Iterable> partitionNameBatches = partitionExponentially(hivePartitions, minPartitionBatchSize, maxPartitionBatchSize); Iterable> partitionBatches = transform(partitionNameBatches, partitionBatch -> { Map> batch = metastore.getPartitionsByNames( + new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName(), Lists.transform(partitionBatch, HivePartition::getPartitionId)); diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/SyncPartitionMetadataProcedure.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/SyncPartitionMetadataProcedure.java index 6eb401928f96e..f229cc9f65465 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/SyncPartitionMetadataProcedure.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/SyncPartitionMetadataProcedure.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Column; import io.prestosql.plugin.hive.metastore.Partition; import io.prestosql.plugin.hive.metastore.SemiTransactionalHiveMetastore; @@ -104,11 +105,12 @@ public void syncPartitionMetadata(ConnectorSession session, String schemaName, S private void doSyncPartitionMetadata(ConnectorSession session, String schemaName, String tableName, String mode) { SyncMode syncMode = toSyncMode(mode); - HdfsContext context = new HdfsContext(session, schemaName, tableName); + HdfsContext hdfsContext = new HdfsContext(session, schemaName, tableName); + HiveIdentity identity = new HiveIdentity(session); SemiTransactionalHiveMetastore metastore = ((HiveMetadata) hiveMetadataFactory.get()).getMetastore(); SchemaTableName schemaTableName = new SchemaTableName(schemaName, tableName); - Table table = metastore.getTable(schemaName, tableName) + Table table = metastore.getTable(identity, schemaName, tableName) .orElseThrow(() -> new TableNotFoundException(schemaTableName)); if (table.getPartitionColumns().isEmpty()) { throw new PrestoException(INVALID_PROCEDURE_ARGUMENT, "Table is not partitioned: " + schemaTableName); @@ -119,8 +121,8 @@ private void doSyncPartitionMetadata(ConnectorSession session, String schemaName Set partitionsToDrop; try { - FileSystem fileSystem = hdfsEnvironment.getFileSystem(context, tableLocation); - List partitionsInMetastore = metastore.getPartitionNames(schemaName, tableName) + FileSystem fileSystem = hdfsEnvironment.getFileSystem(hdfsContext, tableLocation); + List partitionsInMetastore = metastore.getPartitionNames(identity, schemaName, tableName) .orElseThrow(() -> new TableNotFoundException(schemaTableName)); List partitionsInFileSystem = listDirectory(fileSystem, fileSystem.getFileStatus(tableLocation), table.getPartitionColumns(), table.getPartitionColumns().size()).stream() .map(fileStatus -> fileStatus.getPath().toUri()) diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/authentication/HiveIdentity.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/authentication/HiveIdentity.java new file mode 100644 index 0000000000000..b5ea598c150f5 --- /dev/null +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/authentication/HiveIdentity.java @@ -0,0 +1,85 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.plugin.hive.authentication; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.security.ConnectorIdentity; + +import java.util.Objects; +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public final class HiveIdentity +{ + private static final HiveIdentity NONE_IDENTITY = new HiveIdentity(); + + private final Optional username; + + private HiveIdentity() + { + this.username = Optional.empty(); + } + + public HiveIdentity(ConnectorSession session) + { + this(requireNonNull(session, "session is null").getIdentity()); + } + + public HiveIdentity(ConnectorIdentity identity) + { + requireNonNull(identity, "identity is null"); + this.username = Optional.of(requireNonNull(identity.getUser(), "identity.getUser() is null")); + } + + // this should be called only by CachingHiveMetastore + public static HiveIdentity none() + { + return NONE_IDENTITY; + } + + public Optional getUsername() + { + return username; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("username", username) + .toString(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + HiveIdentity other = (HiveIdentity) o; + return Objects.equals(username, other.username); + } + + @Override + public int hashCode() + { + return Objects.hash(username); + } +} diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/HiveMetastore.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/HiveMetastore.java index a2c22dbe12769..f775d525ff967 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/HiveMetastore.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/HiveMetastore.java @@ -15,6 +15,7 @@ import io.prestosql.plugin.hive.HiveType; import io.prestosql.plugin.hive.PartitionStatistics; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.spi.security.RoleGrant; import io.prestosql.spi.statistics.ColumnStatisticType; import io.prestosql.spi.type.Type; @@ -31,17 +32,17 @@ public interface HiveMetastore List getAllDatabases(); - Optional
getTable(String databaseName, String tableName); + Optional
getTable(HiveIdentity identity, String databaseName, String tableName); Set getSupportedColumnStatistics(Type type); - PartitionStatistics getTableStatistics(String databaseName, String tableName); + PartitionStatistics getTableStatistics(HiveIdentity identity, String databaseName, String tableName); - Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames); + Map getPartitionStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames); - void updateTableStatistics(String databaseName, String tableName, Function update); + void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, Function update); - void updatePartitionStatistics(String databaseName, String tableName, String partitionName, Function update); + void updatePartitionStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, Function update); List getAllTables(String databaseName); @@ -49,46 +50,46 @@ public interface HiveMetastore List getAllViews(String databaseName); - void createDatabase(Database database); + void createDatabase(HiveIdentity identity, Database database); - void dropDatabase(String databaseName); + void dropDatabase(HiveIdentity identity, String databaseName); - void renameDatabase(String databaseName, String newDatabaseName); + void renameDatabase(HiveIdentity identity, String databaseName, String newDatabaseName); - void createTable(Table table, PrincipalPrivileges principalPrivileges); + void createTable(HiveIdentity identity, Table table, PrincipalPrivileges principalPrivileges); - void dropTable(String databaseName, String tableName, boolean deleteData); + void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData); /** * This should only be used if the semantic here is drop and add. Trying to * alter one field of a table object previously acquired from getTable is * probably not what you want. */ - void replaceTable(String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges); + void replaceTable(HiveIdentity identity, String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges); - void renameTable(String databaseName, String tableName, String newDatabaseName, String newTableName); + void renameTable(HiveIdentity identity, String databaseName, String tableName, String newDatabaseName, String newTableName); - void commentTable(String databaseName, String tableName, Optional comment); + void commentTable(HiveIdentity identity, String databaseName, String tableName, Optional comment); - void addColumn(String databaseName, String tableName, String columnName, HiveType columnType, String columnComment); + void addColumn(HiveIdentity identity, String databaseName, String tableName, String columnName, HiveType columnType, String columnComment); - void renameColumn(String databaseName, String tableName, String oldColumnName, String newColumnName); + void renameColumn(HiveIdentity identity, String databaseName, String tableName, String oldColumnName, String newColumnName); - void dropColumn(String databaseName, String tableName, String columnName); + void dropColumn(HiveIdentity identity, String databaseName, String tableName, String columnName); - Optional getPartition(String databaseName, String tableName, List partitionValues); + Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues); - Optional> getPartitionNames(String databaseName, String tableName); + Optional> getPartitionNames(HiveIdentity identity, String databaseName, String tableName); - Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts); + Optional> getPartitionNamesByParts(HiveIdentity identity, String databaseName, String tableName, List parts); - Map> getPartitionsByNames(String databaseName, String tableName, List partitionNames); + Map> getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames); - void addPartitions(String databaseName, String tableName, List partitions); + void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitions); - void dropPartition(String databaseName, String tableName, List parts, boolean deleteData); + void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData); - void alterPartition(String databaseName, String tableName, PartitionWithStatistics partition); + void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partition); void createRole(String role, String grantor); @@ -107,4 +108,6 @@ public interface HiveMetastore void revokeTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, Set privileges); Set listTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal principal); + + boolean isImpersonationEnabled(); } diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/HivePageSinkMetadataProvider.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/HivePageSinkMetadataProvider.java index ab24670dc6da0..e569f243ad5bc 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/HivePageSinkMetadataProvider.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/HivePageSinkMetadataProvider.java @@ -13,6 +13,7 @@ */ package io.prestosql.plugin.hive.metastore; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.spi.connector.SchemaTableName; import java.util.List; @@ -24,15 +25,17 @@ public class HivePageSinkMetadataProvider { + private final HiveIdentity identity; private final HiveMetastore delegate; private final SchemaTableName schemaTableName; private final Optional
table; private final Map, Optional> modifiedPartitions; - public HivePageSinkMetadataProvider(HivePageSinkMetadata pageSinkMetadata, HiveMetastore delegate) + public HivePageSinkMetadataProvider(HivePageSinkMetadata pageSinkMetadata, HiveMetastore delegate, HiveIdentity identity) { requireNonNull(pageSinkMetadata, "pageSinkMetadata is null"); this.delegate = delegate; + this.identity = requireNonNull(identity, "identity is null"); this.schemaTableName = pageSinkMetadata.getSchemaTableName(); this.table = pageSinkMetadata.getTable(); this.modifiedPartitions = pageSinkMetadata.getModifiedPartitions(); @@ -51,7 +54,7 @@ public Optional getPartition(List partitionValues) } Optional modifiedPartition = modifiedPartitions.get(partitionValues); if (modifiedPartition == null) { - return delegate.getPartition(schemaTableName.getSchemaName(), schemaTableName.getTableName(), partitionValues); + return delegate.getPartition(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName(), partitionValues); } else { return modifiedPartition; diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/MetastoreUtil.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/MetastoreUtil.java index c2c0714cdc189..81a7fcef24f92 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/MetastoreUtil.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/MetastoreUtil.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMultimap; import io.prestosql.plugin.hive.PartitionOfflineException; import io.prestosql.plugin.hive.TableOfflineException; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.spi.PrestoException; import io.prestosql.spi.connector.SchemaTableName; import io.prestosql.spi.connector.TableNotFoundException; @@ -250,9 +251,9 @@ public static void verifyOnline(SchemaTableName tableName, Optional part } } - public static void verifyCanDropColumn(HiveMetastore metastore, String databaseName, String tableName, String columnName) + public static void verifyCanDropColumn(HiveMetastore metastore, HiveIdentity identity, String databaseName, String tableName, String columnName) { - Table table = metastore.getTable(databaseName, tableName) + Table table = metastore.getTable(identity, databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); if (table.getPartitionColumns().stream().anyMatch(column -> column.getName().equals(columnName))) { 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 94e6b2470dd86..2140044876b56 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 @@ -25,6 +25,7 @@ import io.prestosql.plugin.hive.HiveConfig; import io.prestosql.plugin.hive.HiveType; import io.prestosql.plugin.hive.PartitionStatistics; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.spi.PrestoException; import io.prestosql.spi.security.RoleGrant; import io.prestosql.spi.statistics.ColumnStatisticType; @@ -208,9 +209,9 @@ public List getAllDatabases() } @Override - public Optional
getTable(String databaseName, String tableName) + public Optional
getTable(HiveIdentity identity, String databaseName, String tableName) { - return loadValue(tableCache, hiveTableName(databaseName, tableName), () -> delegate.getTable(databaseName, tableName)); + return loadValue(tableCache, hiveTableName(databaseName, tableName), () -> delegate.getTable(identity, databaseName, tableName)); } @Override @@ -220,35 +221,35 @@ public Set getSupportedColumnStatistics(Type type) } @Override - public PartitionStatistics getTableStatistics(String databaseName, String tableName) + public PartitionStatistics getTableStatistics(HiveIdentity identity, String databaseName, String tableName) { return loadValue( tableStatisticsCache, hiveTableName(databaseName, tableName), - () -> delegate.getTableStatistics(databaseName, tableName)); + () -> delegate.getTableStatistics(identity, databaseName, tableName)); } @Override - public Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames) + public Map getPartitionStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames) { return loadValue( partitionStatisticsCache, getHivePartitionNames(databaseName, tableName, partitionNames), - () -> delegate.getPartitionStatistics(databaseName, tableName, partitionNames)); + () -> delegate.getPartitionStatistics(identity, databaseName, tableName, partitionNames)); } @Override - public void updateTableStatistics(String databaseName, String tableName, Function update) + public void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, Function update) { verifyRecordingMode(); - delegate.updateTableStatistics(databaseName, tableName, update); + delegate.updateTableStatistics(identity, databaseName, tableName, update); } @Override - public void updatePartitionStatistics(String databaseName, String tableName, String partitionName, Function update) + public void updatePartitionStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, Function update) { verifyRecordingMode(); - delegate.updatePartitionStatistics(databaseName, tableName, partitionName, update); + delegate.updatePartitionStatistics(identity, databaseName, tableName, partitionName, update); } @Override @@ -271,137 +272,137 @@ public List getAllViews(String databaseName) } @Override - public void createDatabase(Database database) + public void createDatabase(HiveIdentity identity, Database database) { verifyRecordingMode(); - delegate.createDatabase(database); + delegate.createDatabase(identity, database); } @Override - public void dropDatabase(String databaseName) + public void dropDatabase(HiveIdentity identity, String databaseName) { verifyRecordingMode(); - delegate.dropDatabase(databaseName); + delegate.dropDatabase(identity, databaseName); } @Override - public void renameDatabase(String databaseName, String newDatabaseName) + public void renameDatabase(HiveIdentity identity, String databaseName, String newDatabaseName) { verifyRecordingMode(); - delegate.renameDatabase(databaseName, newDatabaseName); + delegate.renameDatabase(identity, databaseName, newDatabaseName); } @Override - public void createTable(Table table, PrincipalPrivileges principalPrivileges) + public void createTable(HiveIdentity identity, Table table, PrincipalPrivileges principalPrivileges) { verifyRecordingMode(); - delegate.createTable(table, principalPrivileges); + delegate.createTable(identity, table, principalPrivileges); } @Override - public void dropTable(String databaseName, String tableName, boolean deleteData) + public void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) { verifyRecordingMode(); - delegate.dropTable(databaseName, tableName, deleteData); + delegate.dropTable(identity, databaseName, tableName, deleteData); } @Override - public void replaceTable(String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) + public void replaceTable(HiveIdentity identity, String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) { verifyRecordingMode(); - delegate.replaceTable(databaseName, tableName, newTable, principalPrivileges); + delegate.replaceTable(identity, databaseName, tableName, newTable, principalPrivileges); } @Override - public void renameTable(String databaseName, String tableName, String newDatabaseName, String newTableName) + public void renameTable(HiveIdentity identity, String databaseName, String tableName, String newDatabaseName, String newTableName) { verifyRecordingMode(); - delegate.renameTable(databaseName, tableName, newDatabaseName, newTableName); + delegate.renameTable(identity, databaseName, tableName, newDatabaseName, newTableName); } @Override - public void commentTable(String databaseName, String tableName, Optional comment) + public void commentTable(HiveIdentity identity, String databaseName, String tableName, Optional comment) { verifyRecordingMode(); - delegate.commentTable(databaseName, tableName, comment); + delegate.commentTable(identity, databaseName, tableName, comment); } @Override - public void addColumn(String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) + public void addColumn(HiveIdentity identity, String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) { verifyRecordingMode(); - delegate.addColumn(databaseName, tableName, columnName, columnType, columnComment); + delegate.addColumn(identity, databaseName, tableName, columnName, columnType, columnComment); } @Override - public void renameColumn(String databaseName, String tableName, String oldColumnName, String newColumnName) + public void renameColumn(HiveIdentity identity, String databaseName, String tableName, String oldColumnName, String newColumnName) { verifyRecordingMode(); - delegate.renameColumn(databaseName, tableName, oldColumnName, newColumnName); + delegate.renameColumn(identity, databaseName, tableName, oldColumnName, newColumnName); } @Override - public void dropColumn(String databaseName, String tableName, String columnName) + public void dropColumn(HiveIdentity identity, String databaseName, String tableName, String columnName) { verifyRecordingMode(); - delegate.dropColumn(databaseName, tableName, columnName); + delegate.dropColumn(identity, databaseName, tableName, columnName); } @Override - public Optional getPartition(String databaseName, String tableName, List partitionValues) + public Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues) { return loadValue( partitionCache, hivePartitionName(databaseName, tableName, partitionValues), - () -> delegate.getPartition(databaseName, tableName, partitionValues)); + () -> delegate.getPartition(identity, databaseName, tableName, partitionValues)); } @Override - public Optional> getPartitionNames(String databaseName, String tableName) + public Optional> getPartitionNames(HiveIdentity identity, String databaseName, String tableName) { return loadValue( partitionNamesCache, hiveTableName(databaseName, tableName), - () -> delegate.getPartitionNames(databaseName, tableName)); + () -> delegate.getPartitionNames(identity, databaseName, tableName)); } @Override - public Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) + public Optional> getPartitionNamesByParts(HiveIdentity identity, String databaseName, String tableName, List parts) { return loadValue( partitionNamesByPartsCache, partitionFilter(databaseName, tableName, parts), - () -> delegate.getPartitionNamesByParts(databaseName, tableName, parts)); + () -> delegate.getPartitionNamesByParts(identity, databaseName, tableName, parts)); } @Override - public Map> getPartitionsByNames(String databaseName, String tableName, List partitionNames) + public Map> getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames) { return loadValue( partitionsByNamesCache, getHivePartitionNames(databaseName, tableName, ImmutableSet.copyOf(partitionNames)), - () -> delegate.getPartitionsByNames(databaseName, tableName, partitionNames)); + () -> delegate.getPartitionsByNames(identity, databaseName, tableName, partitionNames)); } @Override - public void addPartitions(String databaseName, String tableName, List partitions) + public void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitions) { verifyRecordingMode(); - delegate.addPartitions(databaseName, tableName, partitions); + delegate.addPartitions(identity, databaseName, tableName, partitions); } @Override - public void dropPartition(String databaseName, String tableName, List parts, boolean deleteData) + public void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData) { verifyRecordingMode(); - delegate.dropPartition(databaseName, tableName, parts, deleteData); + delegate.dropPartition(identity, databaseName, tableName, parts, deleteData); } @Override - public void alterPartition(String databaseName, String tableName, PartitionWithStatistics partition) + public void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partition) { verifyRecordingMode(); - delegate.alterPartition(databaseName, tableName, partition); + delegate.alterPartition(identity, databaseName, tableName, partition); } @Override @@ -483,6 +484,12 @@ public Set listRoleGrants(HivePrincipal principal) () -> delegate.listRoleGrants(principal)); } + @Override + public boolean isImpersonationEnabled() + { + return delegate.isImpersonationEnabled(); + } + private V loadValue(Cache cache, K key, Supplier valueSupplier) { if (replay) { 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 d986738eea18e..19e1a1261be4b 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 @@ -30,6 +30,7 @@ import io.prestosql.plugin.hive.PartitionNotFoundException; import io.prestosql.plugin.hive.PartitionStatistics; import io.prestosql.plugin.hive.TableAlreadyExistsException; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.spi.PrestoException; import io.prestosql.spi.StandardErrorCode; import io.prestosql.spi.connector.ConnectorSession; @@ -144,12 +145,12 @@ public synchronized List getAllTables(String databaseName) return delegate.getAllTables(databaseName); } - public synchronized Optional
getTable(String databaseName, String tableName) + public synchronized Optional
getTable(HiveIdentity identity, String databaseName, String tableName) { checkReadable(); Action tableAction = tableActions.get(new SchemaTableName(databaseName, tableName)); if (tableAction == null) { - return delegate.getTable(databaseName, tableName); + return delegate.getTable(identity, databaseName, tableName); } switch (tableAction.getType()) { case ADD: @@ -168,12 +169,12 @@ public synchronized Set getSupportedColumnStatistics(Type t return delegate.getSupportedColumnStatistics(type); } - public synchronized PartitionStatistics getTableStatistics(String databaseName, String tableName) + public synchronized PartitionStatistics getTableStatistics(HiveIdentity identity, String databaseName, String tableName) { checkReadable(); Action tableAction = tableActions.get(new SchemaTableName(databaseName, tableName)); if (tableAction == null) { - return delegate.getTableStatistics(databaseName, tableName); + return delegate.getTableStatistics(identity, databaseName, tableName); } switch (tableAction.getType()) { case ADD: @@ -187,10 +188,10 @@ public synchronized PartitionStatistics getTableStatistics(String databaseName, } } - public synchronized Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames) + public synchronized Map getPartitionStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames) { checkReadable(); - Optional
table = getTable(databaseName, tableName); + Optional
table = getTable(identity, databaseName, tableName); if (!table.isPresent()) { return ImmutableMap.of(); } @@ -218,7 +219,7 @@ public synchronized Map getPartitionStatistics(Stri } } - Map delegateResult = delegate.getPartitionStatistics(databaseName, tableName, partitionNamesToQuery.build()); + Map delegateResult = delegate.getPartitionStatistics(identity, databaseName, tableName, partitionNamesToQuery.build()); if (!delegateResult.isEmpty()) { resultBuilder.putAll(delegateResult); } @@ -254,10 +255,10 @@ private TableSource getTableSource(String databaseName, String tableName) } } - public synchronized HivePageSinkMetadata generatePageSinkMetadata(SchemaTableName schemaTableName) + public synchronized HivePageSinkMetadata generatePageSinkMetadata(HiveIdentity identity, SchemaTableName schemaTableName) { checkReadable(); - Optional
table = getTable(schemaTableName.getSchemaName(), schemaTableName.getTableName()); + Optional
table = getTable(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName()); if (!table.isPresent()) { return new HivePageSinkMetadata(schemaTableName, Optional.empty(), ImmutableMap.of()); } @@ -288,34 +289,35 @@ public synchronized List getAllViews(String databaseName) return delegate.getAllViews(databaseName); } - public synchronized void createDatabase(Database database) + public synchronized void createDatabase(HiveIdentity identity, Database database) { - setExclusive((delegate, hdfsEnvironment) -> delegate.createDatabase(database)); + setExclusive((delegate, hdfsEnvironment) -> delegate.createDatabase(identity, database)); } - public synchronized void dropDatabase(String schemaName) + public synchronized void dropDatabase(HiveIdentity identity, String schemaName) { - setExclusive((delegate, hdfsEnvironment) -> delegate.dropDatabase(schemaName)); + setExclusive((delegate, hdfsEnvironment) -> delegate.dropDatabase(identity, schemaName)); } - public synchronized void renameDatabase(String source, String target) + public synchronized void renameDatabase(HiveIdentity identity, String source, String target) { - setExclusive((delegate, hdfsEnvironment) -> delegate.renameDatabase(source, target)); + setExclusive((delegate, hdfsEnvironment) -> delegate.renameDatabase(identity, source, target)); } // TODO: Allow updating statistics for 2 tables in the same transaction - public synchronized void setTableStatistics(Table table, PartitionStatistics tableStatistics) + public synchronized void setTableStatistics(HiveIdentity identity, Table table, PartitionStatistics tableStatistics) { setExclusive((delegate, hdfsEnvironment) -> - delegate.updateTableStatistics(table.getDatabaseName(), table.getTableName(), statistics -> updatePartitionStatistics(statistics, tableStatistics))); + delegate.updateTableStatistics(identity, table.getDatabaseName(), table.getTableName(), statistics -> updatePartitionStatistics(statistics, tableStatistics))); } // TODO: Allow updating statistics for 2 tables in the same transaction - public synchronized void setPartitionStatistics(Table table, Map, PartitionStatistics> partitionStatisticsMap) + public synchronized void setPartitionStatistics(HiveIdentity identity, Table table, Map, PartitionStatistics> partitionStatisticsMap) { setExclusive((delegate, hdfsEnvironment) -> partitionStatisticsMap.forEach((partitionValues, newPartitionStats) -> delegate.updatePartitionStatistics( + identity, table.getDatabaseName(), table.getTableName(), getPartitionName(table, partitionValues), @@ -357,19 +359,20 @@ public synchronized void createTable( // When creating a table, it should never have partition actions. This is just a sanity check. checkNoPartitionAction(table.getDatabaseName(), table.getTableName()); Action oldTableAction = tableActions.get(table.getSchemaTableName()); - TableAndMore tableAndMore = new TableAndMore(table, Optional.of(principalPrivileges), currentPath, Optional.empty(), ignoreExisting, statistics, statistics); + HiveIdentity identity = new HiveIdentity(session); + TableAndMore tableAndMore = new TableAndMore(table, identity, Optional.of(principalPrivileges), currentPath, Optional.empty(), ignoreExisting, statistics, statistics); if (oldTableAction == null) { - HdfsContext context = new HdfsContext(session, table.getDatabaseName(), table.getTableName()); - tableActions.put(table.getSchemaTableName(), new Action<>(ActionType.ADD, tableAndMore, context)); + HdfsContext hdfsContext = new HdfsContext(session, table.getDatabaseName(), table.getTableName()); + tableActions.put(table.getSchemaTableName(), new Action<>(ActionType.ADD, tableAndMore, hdfsContext, identity)); return; } switch (oldTableAction.getType()) { case DROP: - if (!oldTableAction.getContext().getIdentity().getUser().equals(session.getUser())) { + if (!oldTableAction.getHdfsContext().getIdentity().getUser().equals(session.getUser())) { throw new PrestoException(TRANSACTION_CONFLICT, "Operation on the same table with different user in the same transaction is not supported"); } - HdfsContext context = new HdfsContext(session, table.getDatabaseName(), table.getTableName()); - tableActions.put(table.getSchemaTableName(), new Action<>(ActionType.ALTER, tableAndMore, context)); + HdfsContext hdfsContext = new HdfsContext(session, table.getDatabaseName(), table.getTableName()); + tableActions.put(table.getSchemaTableName(), new Action<>(ActionType.ALTER, tableAndMore, hdfsContext, identity)); break; case ADD: case ALTER: @@ -388,8 +391,9 @@ public synchronized void dropTable(ConnectorSession session, String databaseName SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); Action oldTableAction = tableActions.get(schemaTableName); if (oldTableAction == null || oldTableAction.getType() == ActionType.ALTER) { - HdfsContext context = new HdfsContext(session, databaseName, tableName); - tableActions.put(schemaTableName, new Action<>(ActionType.DROP, null, context)); + HdfsContext hdfsContext = new HdfsContext(session, databaseName, tableName); + HiveIdentity identity = new HiveIdentity(session); + tableActions.put(schemaTableName, new Action<>(ActionType.DROP, null, hdfsContext, identity)); return; } switch (oldTableAction.getType()) { @@ -404,34 +408,34 @@ public synchronized void dropTable(ConnectorSession session, String databaseName } } - public synchronized void replaceTable(String databaseName, String tableName, Table table, PrincipalPrivileges principalPrivileges) + public synchronized void replaceTable(HiveIdentity identity, String databaseName, String tableName, Table table, PrincipalPrivileges principalPrivileges) { - setExclusive((delegate, hdfsEnvironment) -> delegate.replaceTable(databaseName, tableName, table, principalPrivileges)); + setExclusive((delegate, hdfsEnvironment) -> delegate.replaceTable(identity, databaseName, tableName, table, principalPrivileges)); } - public synchronized void renameTable(String databaseName, String tableName, String newDatabaseName, String newTableName) + public synchronized void renameTable(HiveIdentity identity, String databaseName, String tableName, String newDatabaseName, String newTableName) { - setExclusive((delegate, hdfsEnvironment) -> delegate.renameTable(databaseName, tableName, newDatabaseName, newTableName)); + setExclusive((delegate, hdfsEnvironment) -> delegate.renameTable(identity, databaseName, tableName, newDatabaseName, newTableName)); } - public synchronized void commentTable(String databaseName, String tableName, Optional comment) + public synchronized void commentTable(HiveIdentity identity, String databaseName, String tableName, Optional comment) { - setExclusive((delegate, hdfsEnvironment) -> delegate.commentTable(databaseName, tableName, comment)); + setExclusive((delegate, hdfsEnvironment) -> delegate.commentTable(identity, databaseName, tableName, comment)); } - public synchronized void addColumn(String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) + public synchronized void addColumn(HiveIdentity identity, String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) { - setExclusive((delegate, hdfsEnvironment) -> delegate.addColumn(databaseName, tableName, columnName, columnType, columnComment)); + setExclusive((delegate, hdfsEnvironment) -> delegate.addColumn(identity, databaseName, tableName, columnName, columnType, columnComment)); } - public synchronized void renameColumn(String databaseName, String tableName, String oldColumnName, String newColumnName) + public synchronized void renameColumn(HiveIdentity identity, String databaseName, String tableName, String oldColumnName, String newColumnName) { - setExclusive((delegate, hdfsEnvironment) -> delegate.renameColumn(databaseName, tableName, oldColumnName, newColumnName)); + setExclusive((delegate, hdfsEnvironment) -> delegate.renameColumn(identity, databaseName, tableName, oldColumnName, newColumnName)); } - public synchronized void dropColumn(String databaseName, String tableName, String columnName) + public synchronized void dropColumn(HiveIdentity identity, String databaseName, String tableName, String columnName) { - setExclusive((delegate, hdfsEnvironment) -> delegate.dropColumn(databaseName, tableName, columnName)); + setExclusive((delegate, hdfsEnvironment) -> delegate.dropColumn(identity, databaseName, tableName, columnName)); } public synchronized void finishInsertIntoExistingTable( @@ -445,26 +449,29 @@ public synchronized void finishInsertIntoExistingTable( // Data can only be inserted into partitions and unpartitioned tables. They can never be inserted into a partitioned table. // Therefore, this method assumes that the table is unpartitioned. setShared(); + HiveIdentity identity = new HiveIdentity(session); SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); Action oldTableAction = tableActions.get(schemaTableName); if (oldTableAction == null) { - Table table = delegate.getTable(databaseName, tableName) + Table table = delegate.getTable(identity, databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(schemaTableName)); - PartitionStatistics currentStatistics = getTableStatistics(databaseName, tableName); - HdfsContext context = new HdfsContext(session, databaseName, tableName); + PartitionStatistics currentStatistics = getTableStatistics(identity, databaseName, tableName); + HdfsContext hdfsContext = new HdfsContext(session, databaseName, tableName); tableActions.put( schemaTableName, new Action<>( ActionType.INSERT_EXISTING, new TableAndMore( table, + identity, Optional.empty(), Optional.of(currentLocation), Optional.of(fileNames), false, merge(currentStatistics, statisticsUpdate), statisticsUpdate), - context)); + hdfsContext, + identity)); return; } @@ -483,7 +490,7 @@ public synchronized void finishInsertIntoExistingTable( public synchronized void truncateUnpartitionedTable(ConnectorSession session, String databaseName, String tableName) { checkReadable(); - Optional
table = getTable(databaseName, tableName); + Optional
table = getTable(new HiveIdentity(session), databaseName, tableName); SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); if (!table.isPresent()) { throw new TableNotFoundException(schemaTableName); @@ -508,23 +515,23 @@ public synchronized void truncateUnpartitionedTable(ConnectorSession session, St }); } - public synchronized Optional> getPartitionNames(String databaseName, String tableName) + public synchronized Optional> getPartitionNames(HiveIdentity identity, String databaseName, String tableName) { - return doGetPartitionNames(databaseName, tableName, Optional.empty()); + return doGetPartitionNames(identity, databaseName, tableName, Optional.empty()); } - public synchronized Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) + public synchronized Optional> getPartitionNamesByParts(HiveIdentity identity, String databaseName, String tableName, List parts) { - return doGetPartitionNames(databaseName, tableName, Optional.of(parts)); + return doGetPartitionNames(identity, databaseName, tableName, Optional.of(parts)); } @GuardedBy("this") - private Optional> doGetPartitionNames(String databaseName, String tableName, Optional> parts) + private Optional> doGetPartitionNames(HiveIdentity identity, String databaseName, String tableName, Optional> parts) { checkHoldsLock(); checkReadable(); - Optional
table = getTable(databaseName, tableName); + Optional
table = getTable(identity, databaseName, tableName); if (!table.isPresent()) { return Optional.empty(); } @@ -537,10 +544,10 @@ private Optional> doGetPartitionNames(String databaseName, String t case PRE_EXISTING_TABLE: { Optional> partitionNameResult; if (parts.isPresent()) { - partitionNameResult = delegate.getPartitionNamesByParts(databaseName, tableName, parts.get()); + partitionNameResult = delegate.getPartitionNamesByParts(identity, databaseName, tableName, parts.get()); } else { - partitionNameResult = delegate.getPartitionNames(databaseName, tableName); + partitionNameResult = delegate.getPartitionNames(identity, databaseName, tableName); } if (!partitionNameResult.isPresent()) { throw new PrestoException(TRANSACTION_CONFLICT, format("Table %s.%s was dropped by another transaction", databaseName, tableName)); @@ -605,7 +612,7 @@ private static boolean partitionValuesMatch(List values, List pa return true; } - public synchronized Optional getPartition(String databaseName, String tableName, List partitionValues) + public synchronized Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues) { checkReadable(); TableSource tableSource = getTableSource(databaseName, tableName); @@ -616,7 +623,7 @@ public synchronized Optional getPartition(String databaseName, String } switch (tableSource) { case PRE_EXISTING_TABLE: - return delegate.getPartition(databaseName, tableName, partitionValues); + return delegate.getPartition(identity, databaseName, tableName, partitionValues); case CREATED_IN_THIS_TRANSACTION: return Optional.empty(); default: @@ -624,7 +631,7 @@ public synchronized Optional getPartition(String databaseName, String } } - public synchronized Map> getPartitionsByNames(String databaseName, String tableName, List partitionNames) + public synchronized Map> getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames) { checkReadable(); TableSource tableSource = getTableSource(databaseName, tableName); @@ -650,7 +657,7 @@ public synchronized Map> getPartitionsByNames(String resultBuilder.put(partitionName, getPartitionFromPartitionAction(partitionAction)); } } - Map> delegateResult = delegate.getPartitionsByNames(databaseName, tableName, partitionNamesToQuery.build()); + Map> delegateResult = delegate.getPartitionsByNames(identity, databaseName, tableName, partitionNamesToQuery.build()); resultBuilder.putAll(delegateResult); return resultBuilder.build(); } @@ -681,21 +688,22 @@ public synchronized void addPartition( checkArgument(getPrestoQueryId(partition).isPresent()); Map, Action> partitionActionsOfTable = partitionActions.computeIfAbsent(new SchemaTableName(databaseName, tableName), k -> new HashMap<>()); Action oldPartitionAction = partitionActionsOfTable.get(partition.getValues()); - HdfsContext context = new HdfsContext(session, databaseName, tableName); + HdfsContext hdfsContext = new HdfsContext(session, databaseName, tableName); + HiveIdentity identity = new HiveIdentity(session); if (oldPartitionAction == null) { partitionActionsOfTable.put( partition.getValues(), - new Action<>(ActionType.ADD, new PartitionAndMore(partition, currentLocation, Optional.empty(), statistics, statistics), context)); + new Action<>(ActionType.ADD, new PartitionAndMore(identity, partition, currentLocation, Optional.empty(), statistics, statistics), hdfsContext, identity)); return; } switch (oldPartitionAction.getType()) { case DROP: { - if (!oldPartitionAction.getContext().getIdentity().getUser().equals(session.getUser())) { + if (!oldPartitionAction.getHdfsContext().getIdentity().getUser().equals(session.getUser())) { throw new PrestoException(TRANSACTION_CONFLICT, "Operation on the same partition with different user in the same transaction is not supported"); } partitionActionsOfTable.put( partition.getValues(), - new Action<>(ActionType.ALTER, new PartitionAndMore(partition, currentLocation, Optional.empty(), statistics, statistics), context)); + new Action<>(ActionType.ALTER, new PartitionAndMore(identity, partition, currentLocation, Optional.empty(), statistics, statistics), hdfsContext, identity)); break; } case ADD: @@ -713,8 +721,9 @@ public synchronized void dropPartition(ConnectorSession session, String database Map, Action> partitionActionsOfTable = partitionActions.computeIfAbsent(new SchemaTableName(databaseName, tableName), k -> new HashMap<>()); Action oldPartitionAction = partitionActionsOfTable.get(partitionValues); if (oldPartitionAction == null) { - HdfsContext context = new HdfsContext(session, databaseName, tableName); - partitionActionsOfTable.put(partitionValues, new Action<>(ActionType.DROP, null, context)); + HdfsContext hdfsContext = new HdfsContext(session, databaseName, tableName); + HiveIdentity identity = new HiveIdentity(session); + partitionActionsOfTable.put(partitionValues, new Action<>(ActionType.DROP, null, hdfsContext, identity)); return; } switch (oldPartitionAction.getType()) { @@ -741,14 +750,15 @@ public synchronized void finishInsertIntoExistingPartition( PartitionStatistics statisticsUpdate) { setShared(); + HiveIdentity identity = new HiveIdentity(session); SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); Map, Action> partitionActionsOfTable = partitionActions.computeIfAbsent(schemaTableName, k -> new HashMap<>()); Action oldPartitionAction = partitionActionsOfTable.get(partitionValues); if (oldPartitionAction == null) { - Partition partition = delegate.getPartition(databaseName, tableName, partitionValues) + Partition partition = delegate.getPartition(identity, databaseName, tableName, partitionValues) .orElseThrow(() -> new PartitionNotFoundException(schemaTableName, partitionValues)); - String partitionName = getPartitionName(databaseName, tableName, partitionValues); - PartitionStatistics currentStatistics = delegate.getPartitionStatistics(databaseName, tableName, ImmutableSet.of(partitionName)).get(partitionName); + String partitionName = getPartitionName(identity, databaseName, tableName, partitionValues); + PartitionStatistics currentStatistics = delegate.getPartitionStatistics(identity, databaseName, tableName, ImmutableSet.of(partitionName)).get(partitionName); if (currentStatistics == null) { throw new PrestoException(HIVE_METASTORE_ERROR, "currentStatistics is null"); } @@ -758,12 +768,14 @@ public synchronized void finishInsertIntoExistingPartition( new Action<>( ActionType.INSERT_EXISTING, new PartitionAndMore( + identity, partition, currentLocation, Optional.of(fileNames), merge(currentStatistics, statisticsUpdate), statisticsUpdate), - context)); + context, + identity)); return; } @@ -779,9 +791,9 @@ public synchronized void finishInsertIntoExistingPartition( } } - private String getPartitionName(String databaseName, String tableName, List partitionValues) + private String getPartitionName(HiveIdentity identity, String databaseName, String tableName, List partitionValues) { - Table table = getTable(databaseName, tableName) + Table table = getTable(identity, databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); return getPartitionName(table, partitionValues); } @@ -826,13 +838,13 @@ public synchronized Set listRoleGrants(HivePrincipal principal) return delegate.listRoleGrants(principal); } - public synchronized Set listTablePrivileges(String databaseName, String tableName, HivePrincipal principal) + public synchronized Set listTablePrivileges(HiveIdentity identity, String databaseName, String tableName, HivePrincipal principal) { checkReadable(); SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); Action tableAction = tableActions.get(schemaTableName); if (tableAction == null) { - return delegate.listTablePrivileges(databaseName, tableName, getTableOwner(databaseName, tableName), principal); + return delegate.listTablePrivileges(databaseName, tableName, getTableOwner(identity, databaseName, tableName), principal); } switch (tableAction.getType()) { case ADD: @@ -850,7 +862,7 @@ public synchronized Set listTablePrivileges(String databaseNa .build(); } case INSERT_EXISTING: - return delegate.listTablePrivileges(databaseName, tableName, getTableOwner(databaseName, tableName), principal); + return delegate.listTablePrivileges(databaseName, tableName, getTableOwner(identity, databaseName, tableName), principal); case DROP: throw new TableNotFoundException(schemaTableName); default: @@ -858,21 +870,21 @@ public synchronized Set listTablePrivileges(String databaseNa } } - private String getTableOwner(String databaseName, String tableName) + private String getTableOwner(HiveIdentity identity, String databaseName, String tableName) { - Table table = delegate.getTable(databaseName, tableName) + Table table = delegate.getTable(identity, databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); return table.getOwner(); } - public synchronized void grantTablePrivileges(String databaseName, String tableName, HivePrincipal grantee, Set privileges) + public synchronized void grantTablePrivileges(HiveIdentity identity, String databaseName, String tableName, HivePrincipal grantee, Set privileges) { - setExclusive((delegate, hdfsEnvironment) -> delegate.grantTablePrivileges(databaseName, tableName, getTableOwner(databaseName, tableName), grantee, privileges)); + setExclusive((delegate, hdfsEnvironment) -> delegate.grantTablePrivileges(databaseName, tableName, getTableOwner(identity, databaseName, tableName), grantee, privileges)); } - public synchronized void revokeTablePrivileges(String databaseName, String tableName, HivePrincipal grantee, Set privileges) + public synchronized void revokeTablePrivileges(HiveIdentity identity, String databaseName, String tableName, HivePrincipal grantee, Set privileges) { - setExclusive((delegate, hdfsEnvironment) -> delegate.revokeTablePrivileges(databaseName, tableName, getTableOwner(databaseName, tableName), grantee, privileges)); + setExclusive((delegate, hdfsEnvironment) -> delegate.revokeTablePrivileges(databaseName, tableName, getTableOwner(identity, databaseName, tableName), grantee, privileges)); } public synchronized void declareIntentionToWrite(ConnectorSession session, WriteMode writeMode, Path stagingPathRoot, SchemaTableName schemaTableName) @@ -884,8 +896,9 @@ public synchronized void declareIntentionToWrite(ConnectorSession session, Write throw new PrestoException(NOT_SUPPORTED, "Can not insert into a table with a partition that has been modified in the same transaction when Presto is configured to skip temporary directories."); } } - HdfsContext context = new HdfsContext(session, schemaTableName.getSchemaName(), schemaTableName.getTableName()); - declaredIntentionsToWrite.add(new DeclaredIntentionToWrite(writeMode, context, session.getQueryId(), stagingPathRoot, schemaTableName)); + HdfsContext hdfsContext = new HdfsContext(session, schemaTableName.getSchemaName(), schemaTableName.getTableName()); + HiveIdentity identity = new HiveIdentity(session); + declaredIntentionsToWrite.add(new DeclaredIntentionToWrite(writeMode, hdfsContext, identity, session.getQueryId(), stagingPathRoot, schemaTableName)); } public synchronized void commit() @@ -945,16 +958,16 @@ private void commitShared() Action action = entry.getValue(); switch (action.getType()) { case DROP: - committer.prepareDropTable(schemaTableName); + committer.prepareDropTable(action.getIdentity(), schemaTableName); break; case ALTER: - committer.prepareAlterTable(action.getContext(), action.getData()); + committer.prepareAlterTable(action.getHdfsContext(), action.getIdentity(), action.getData()); break; case ADD: - committer.prepareAddTable(action.getContext(), action.getData()); + committer.prepareAddTable(action.getHdfsContext(), action.getData()); break; case INSERT_EXISTING: - committer.prepareInsertExistingTable(action.getContext(), action.getData()); + committer.prepareInsertExistingTable(action.getHdfsContext(), action.getData()); break; default: throw new IllegalStateException("Unknown action type"); @@ -967,16 +980,16 @@ private void commitShared() Action action = partitionEntry.getValue(); switch (action.getType()) { case DROP: - committer.prepareDropPartition(schemaTableName, partitionValues); + committer.prepareDropPartition(action.getIdentity(), schemaTableName, partitionValues); break; case ALTER: - committer.prepareAlterPartition(action.getContext(), action.getData()); + committer.prepareAlterPartition(action.getHdfsContext(), action.getIdentity(), action.getData()); break; case ADD: - committer.prepareAddPartition(action.getContext(), action.getData()); + committer.prepareAddPartition(action.getHdfsContext(), action.getIdentity(), action.getData()); break; case INSERT_EXISTING: - committer.prepareInsertExistingPartition(action.getContext(), action.getData()); + committer.prepareInsertExistingPartition(action.getHdfsContext(), action.getIdentity(), action.getData()); break; default: throw new IllegalStateException("Unknown action type"); @@ -1070,20 +1083,20 @@ private class Committer // Flag for better error message private boolean deleteOnly = true; - private void prepareDropTable(SchemaTableName schemaTableName) + private void prepareDropTable(HiveIdentity identity, SchemaTableName schemaTableName) { metastoreDeleteOperations.add(new IrreversibleMetastoreOperation( format("drop table %s", schemaTableName), - () -> delegate.dropTable(schemaTableName.getSchemaName(), schemaTableName.getTableName(), true))); + () -> delegate.dropTable(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName(), true))); } - private void prepareAlterTable(HdfsContext context, TableAndMore tableAndMore) + private void prepareAlterTable(HdfsContext hdfsContext, HiveIdentity identity, TableAndMore tableAndMore) { deleteOnly = false; Table table = tableAndMore.getTable(); String targetLocation = table.getStorage().getLocation(); - Table oldTable = delegate.getTable(table.getDatabaseName(), table.getTableName()) + Table oldTable = delegate.getTable(identity, table.getDatabaseName(), table.getTableName()) .orElseThrow(() -> new PrestoException(TRANSACTION_CONFLICT, "The table that this transaction modified was deleted in another transaction. " + table.getSchemaTableName())); String oldTableLocation = oldTable.getStorage().getLocation(); Path oldTablePath = new Path(oldTableLocation); @@ -1096,21 +1109,21 @@ private void prepareAlterTable(HdfsContext context, TableAndMore tableAndMore) // Otherwise, // * Remember we will need to delete the location of the old partition at the end if transaction successfully commits if (targetLocation.equals(oldTableLocation)) { - String queryId = context.getQueryId().orElseThrow(() -> new IllegalArgumentException("query ID not present")); + String queryId = hdfsContext.getQueryId().orElseThrow(() -> new IllegalArgumentException("query ID not present")); Path oldTableStagingPath = new Path(oldTablePath.getParent(), "_temp_" + oldTablePath.getName() + "_" + queryId); renameDirectory( - context, + hdfsContext, hdfsEnvironment, oldTablePath, oldTableStagingPath, - () -> renameTasksForAbort.add(new DirectoryRenameTask(context, oldTableStagingPath, oldTablePath))); + () -> renameTasksForAbort.add(new DirectoryRenameTask(hdfsContext, oldTableStagingPath, oldTablePath))); if (!skipDeletionForAlter) { - deletionTasksForFinish.add(new DirectoryDeletionTask(context, oldTableStagingPath)); + deletionTasksForFinish.add(new DirectoryDeletionTask(hdfsContext, oldTableStagingPath)); } } else { if (!skipDeletionForAlter) { - deletionTasksForFinish.add(new DirectoryDeletionTask(context, oldTablePath)); + deletionTasksForFinish.add(new DirectoryDeletionTask(hdfsContext, oldTablePath)); } } @@ -1119,17 +1132,18 @@ private void prepareAlterTable(HdfsContext context, TableAndMore tableAndMore) Path targetPath = new Path(targetLocation); if (!targetPath.equals(currentPath)) { renameDirectory( - context, + hdfsContext, hdfsEnvironment, currentPath, targetPath, - () -> cleanUpTasksForAbort.add(new DirectoryCleanUpTask(context, targetPath, true))); + () -> cleanUpTasksForAbort.add(new DirectoryCleanUpTask(hdfsContext, targetPath, true))); } // Partition alter must happen regardless of whether original and current location is the same // because metadata might change: e.g. storage format, column types, etc - alterTableOperations.add(new AlterTableOperation(tableAndMore.getTable(), oldTable, tableAndMore.getPrincipalPrivileges())); + alterTableOperations.add(new AlterTableOperation(tableAndMore.getIdentity(), tableAndMore.getTable(), oldTable, tableAndMore.getPrincipalPrivileges())); updateStatisticsOperations.add(new UpdateStatisticsOperation( + tableAndMore.getIdentity(), table.getSchemaTableName(), Optional.empty(), tableAndMore.getStatisticsUpdate(), @@ -1182,9 +1196,10 @@ private void prepareAddTable(HdfsContext context, TableAndMore tableAndMore) } } } - addTableOperations.add(new CreateTableOperation(table, tableAndMore.getPrincipalPrivileges(), tableAndMore.isIgnoreExisting())); + addTableOperations.add(new CreateTableOperation(tableAndMore.getIdentity(), table, tableAndMore.getPrincipalPrivileges(), tableAndMore.isIgnoreExisting())); if (!isPrestoView(table)) { updateStatisticsOperations.add(new UpdateStatisticsOperation( + tableAndMore.getIdentity(), table.getSchemaTableName(), Optional.empty(), tableAndMore.getStatisticsUpdate(), @@ -1204,33 +1219,34 @@ private void prepareInsertExistingTable(HdfsContext context, TableAndMore tableA asyncRename(hdfsEnvironment, renameExecutor, fileRenameCancelled, fileRenameFutures, context, currentPath, targetPath, tableAndMore.getFileNames().get()); } updateStatisticsOperations.add(new UpdateStatisticsOperation( + tableAndMore.getIdentity(), table.getSchemaTableName(), Optional.empty(), tableAndMore.getStatisticsUpdate(), true)); } - private void prepareDropPartition(SchemaTableName schemaTableName, List partitionValues) + private void prepareDropPartition(HiveIdentity identity, SchemaTableName schemaTableName, List partitionValues) { metastoreDeleteOperations.add(new IrreversibleMetastoreOperation( format("drop partition %s.%s %s", schemaTableName.getSchemaName(), schemaTableName.getTableName(), partitionValues), - () -> delegate.dropPartition(schemaTableName.getSchemaName(), schemaTableName.getTableName(), partitionValues, true))); + () -> delegate.dropPartition(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName(), partitionValues, true))); } - private void prepareAlterPartition(HdfsContext context, PartitionAndMore partitionAndMore) + private void prepareAlterPartition(HdfsContext hdfsContext, HiveIdentity identity, PartitionAndMore partitionAndMore) { deleteOnly = false; Partition partition = partitionAndMore.getPartition(); String targetLocation = partition.getStorage().getLocation(); - Optional oldPartition = delegate.getPartition(partition.getDatabaseName(), partition.getTableName(), partition.getValues()); + Optional oldPartition = delegate.getPartition(identity, partition.getDatabaseName(), partition.getTableName(), partition.getValues()); if (!oldPartition.isPresent()) { throw new PrestoException( TRANSACTION_CONFLICT, format("The partition that this transaction modified was deleted in another transaction. %s %s", partition.getTableName(), partition.getValues())); } - String partitionName = getPartitionName(partition.getDatabaseName(), partition.getTableName(), partition.getValues()); - PartitionStatistics oldPartitionStatistics = getExistingPartitionStatistics(partition, partitionName); + String partitionName = getPartitionName(identity, partition.getDatabaseName(), partition.getTableName(), partition.getValues()); + PartitionStatistics oldPartitionStatistics = getExistingPartitionStatistics(identity, partition, partitionName); String oldPartitionLocation = oldPartition.get().getStorage().getLocation(); Path oldPartitionPath = new Path(oldPartitionLocation); @@ -1242,21 +1258,21 @@ private void prepareAlterPartition(HdfsContext context, PartitionAndMore partiti // Otherwise, // * Remember we will need to delete the location of the old partition at the end if transaction successfully commits if (targetLocation.equals(oldPartitionLocation)) { - String queryId = context.getQueryId().orElseThrow(() -> new IllegalArgumentException("query ID not present")); + String queryId = hdfsContext.getQueryId().orElseThrow(() -> new IllegalArgumentException("query ID not present")); Path oldPartitionStagingPath = new Path(oldPartitionPath.getParent(), "_temp_" + oldPartitionPath.getName() + "_" + queryId); renameDirectory( - context, + hdfsContext, hdfsEnvironment, oldPartitionPath, oldPartitionStagingPath, - () -> renameTasksForAbort.add(new DirectoryRenameTask(context, oldPartitionStagingPath, oldPartitionPath))); + () -> renameTasksForAbort.add(new DirectoryRenameTask(hdfsContext, oldPartitionStagingPath, oldPartitionPath))); if (!skipDeletionForAlter) { - deletionTasksForFinish.add(new DirectoryDeletionTask(context, oldPartitionStagingPath)); + deletionTasksForFinish.add(new DirectoryDeletionTask(hdfsContext, oldPartitionStagingPath)); } } else { if (!skipDeletionForAlter) { - deletionTasksForFinish.add(new DirectoryDeletionTask(context, oldPartitionPath)); + deletionTasksForFinish.add(new DirectoryDeletionTask(hdfsContext, oldPartitionPath)); } } @@ -1264,23 +1280,24 @@ private void prepareAlterPartition(HdfsContext context, PartitionAndMore partiti Path targetPath = new Path(targetLocation); if (!targetPath.equals(currentPath)) { renameDirectory( - context, + hdfsContext, hdfsEnvironment, currentPath, targetPath, - () -> cleanUpTasksForAbort.add(new DirectoryCleanUpTask(context, targetPath, true))); + () -> cleanUpTasksForAbort.add(new DirectoryCleanUpTask(hdfsContext, targetPath, true))); } // Partition alter must happen regardless of whether original and current location is the same // because metadata might change: e.g. storage format, column types, etc alterPartitionOperations.add(new AlterPartitionOperation( + partitionAndMore.getIdentity(), new PartitionWithStatistics(partition, partitionName, partitionAndMore.getStatisticsUpdate()), new PartitionWithStatistics(oldPartition.get(), partitionName, oldPartitionStatistics))); } - private PartitionStatistics getExistingPartitionStatistics(Partition partition, String partitionName) + private PartitionStatistics getExistingPartitionStatistics(HiveIdentity identity, Partition partition, String partitionName) { try { - PartitionStatistics statistics = delegate.getPartitionStatistics(partition.getDatabaseName(), partition.getTableName(), ImmutableSet.of(partitionName)) + PartitionStatistics statistics = delegate.getPartitionStatistics(identity, partition.getDatabaseName(), partition.getTableName(), ImmutableSet.of(partitionName)) .get(partitionName); if (statistics == null) { throw new PrestoException( @@ -1303,7 +1320,7 @@ private PartitionStatistics getExistingPartitionStatistics(Partition partition, } } - private void prepareAddPartition(HdfsContext context, PartitionAndMore partitionAndMore) + private void prepareAddPartition(HdfsContext hdfsContext, HiveIdentity identity, PartitionAndMore partitionAndMore) { deleteOnly = false; @@ -1314,40 +1331,41 @@ private void prepareAddPartition(HdfsContext context, PartitionAndMore partition PartitionAdder partitionAdder = partitionAdders.computeIfAbsent( partition.getSchemaTableName(), - ignored -> new PartitionAdder(partition.getDatabaseName(), partition.getTableName(), delegate, PARTITION_COMMIT_BATCH_SIZE)); + ignored -> new PartitionAdder(partitionAndMore.getIdentity(), partition.getDatabaseName(), partition.getTableName(), delegate, PARTITION_COMMIT_BATCH_SIZE)); - if (pathExists(context, hdfsEnvironment, currentPath)) { + if (pathExists(hdfsContext, hdfsEnvironment, currentPath)) { if (!targetPath.equals(currentPath)) { renameDirectory( - context, + hdfsContext, hdfsEnvironment, currentPath, targetPath, - () -> cleanUpTasksForAbort.add(new DirectoryCleanUpTask(context, targetPath, true))); + () -> cleanUpTasksForAbort.add(new DirectoryCleanUpTask(hdfsContext, targetPath, true))); } } else { - cleanUpTasksForAbort.add(new DirectoryCleanUpTask(context, targetPath, true)); - createDirectory(context, hdfsEnvironment, targetPath); + cleanUpTasksForAbort.add(new DirectoryCleanUpTask(hdfsContext, targetPath, true)); + createDirectory(hdfsContext, hdfsEnvironment, targetPath); } - String partitionName = getPartitionName(partition.getDatabaseName(), partition.getTableName(), partition.getValues()); + String partitionName = getPartitionName(identity, partition.getDatabaseName(), partition.getTableName(), partition.getValues()); partitionAdder.addPartition(new PartitionWithStatistics(partition, partitionName, partitionAndMore.getStatisticsUpdate())); } - private void prepareInsertExistingPartition(HdfsContext context, PartitionAndMore partitionAndMore) + private void prepareInsertExistingPartition(HdfsContext hdfsContext, HiveIdentity identity, PartitionAndMore partitionAndMore) { deleteOnly = false; Partition partition = partitionAndMore.getPartition(); Path targetPath = new Path(partition.getStorage().getLocation()); Path currentPath = partitionAndMore.getCurrentLocation(); - cleanUpTasksForAbort.add(new DirectoryCleanUpTask(context, targetPath, false)); + cleanUpTasksForAbort.add(new DirectoryCleanUpTask(hdfsContext, targetPath, false)); if (!targetPath.equals(currentPath)) { - asyncRename(hdfsEnvironment, renameExecutor, fileRenameCancelled, fileRenameFutures, context, currentPath, targetPath, partitionAndMore.getFileNames()); + asyncRename(hdfsEnvironment, renameExecutor, fileRenameCancelled, fileRenameFutures, hdfsContext, currentPath, targetPath, partitionAndMore.getFileNames()); } updateStatisticsOperations.add(new UpdateStatisticsOperation( + partitionAndMore.getIdentity(), partition.getSchemaTableName(), - Optional.of(getPartitionName(partition.getDatabaseName(), partition.getTableName(), partition.getValues())), + Optional.of(getPartitionName(identity, partition.getDatabaseName(), partition.getTableName(), partition.getValues())), partitionAndMore.getStatisticsUpdate(), true)); } @@ -1394,7 +1412,7 @@ private void deleteEmptyStagingDirectories(List declar continue; } Path path = declaredIntentionToWrite.getRootPath(); - recursiveDeleteFilesAndLog(declaredIntentionToWrite.getContext(), path, ImmutableSet.of(), true, "staging directory cleanup"); + recursiveDeleteFilesAndLog(declaredIntentionToWrite.getHdfsContext(), path, ImmutableSet.of(), true, "staging directory cleanup"); } } @@ -1579,7 +1597,7 @@ private void rollbackShared() // directories must be carried out conservatively. To be safe, we only delete files that start or // end with the query IDs in this transaction. recursiveDeleteFilesAndLog( - declaredIntentionToWrite.getContext(), + declaredIntentionToWrite.getHdfsContext(), rootPath, ImmutableSet.of(declaredIntentionToWrite.getQueryId()), true, @@ -1595,15 +1613,16 @@ private void rollbackShared() Path baseDirectory = declaredIntentionToWrite.getRootPath(); pathsToClean.add(baseDirectory); + HiveIdentity identity = declaredIntentionToWrite.getIdentity(); SchemaTableName schemaTableName = declaredIntentionToWrite.getSchemaTableName(); - Optional
table = delegate.getTable(schemaTableName.getSchemaName(), schemaTableName.getTableName()); + Optional
table = delegate.getTable(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName()); if (table.isPresent()) { // check every existing partition that is outside for the base directory if (!table.get().getPartitionColumns().isEmpty()) { - List partitionNames = delegate.getPartitionNames(schemaTableName.getSchemaName(), schemaTableName.getTableName()) + List partitionNames = delegate.getPartitionNames(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName()) .orElse(ImmutableList.of()); for (List partitionNameBatch : Iterables.partition(partitionNames, 10)) { - Collection> partitions = delegate.getPartitionsByNames(schemaTableName.getSchemaName(), schemaTableName.getTableName(), partitionNameBatch).values(); + Collection> partitions = delegate.getPartitionsByNames(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName(), partitionNameBatch).values(); partitions.stream() .filter(Optional::isPresent) .map(Optional::get) @@ -1626,7 +1645,7 @@ private void rollbackShared() // TODO: It is a known deficiency that some empty directory does not get cleaned up in S3. // We can not delete any of the directories here since we do not know who created them. recursiveDeleteFilesAndLog( - declaredIntentionToWrite.getContext(), + declaredIntentionToWrite.getHdfsContext(), path, ImmutableSet.of(declaredIntentionToWrite.getQueryId()), false, @@ -2004,9 +2023,10 @@ public static class Action { private final ActionType type; private final T data; - private final HdfsContext context; + private final HdfsContext hdfsContext; + private final HiveIdentity identity; - public Action(ActionType type, T data, HdfsContext context) + public Action(ActionType type, T data, HdfsContext hdfsContext, HiveIdentity identity) { this.type = requireNonNull(type, "type is null"); if (type == ActionType.DROP) { @@ -2016,7 +2036,8 @@ public Action(ActionType type, T data, HdfsContext context) requireNonNull(data, "data is null"); } this.data = data; - this.context = requireNonNull(context, "context is null"); + this.hdfsContext = requireNonNull(hdfsContext, "hdfsContext is null"); + this.identity = requireNonNull(identity, "identity is null"); } public ActionType getType() @@ -2030,9 +2051,14 @@ public T getData() return data; } - public HdfsContext getContext() + public HdfsContext getHdfsContext() { - return context; + return hdfsContext; + } + + public HiveIdentity getIdentity() + { + return identity; } @Override @@ -2048,6 +2074,7 @@ public String toString() private static class TableAndMore { private final Table table; + private final HiveIdentity identity; private final Optional principalPrivileges; private final Optional currentLocation; // unpartitioned table only private final Optional> fileNames; @@ -2057,6 +2084,7 @@ private static class TableAndMore public TableAndMore( Table table, + HiveIdentity identity, Optional principalPrivileges, Optional currentLocation, Optional> fileNames, @@ -2065,6 +2093,7 @@ public TableAndMore( PartitionStatistics statisticsUpdate) { this.table = requireNonNull(table, "table is null"); + this.identity = requireNonNull(identity, "identity is null"); this.principalPrivileges = requireNonNull(principalPrivileges, "principalPrivileges is null"); this.currentLocation = requireNonNull(currentLocation, "currentLocation is null"); this.fileNames = requireNonNull(fileNames, "fileNames is null"); @@ -2086,6 +2115,11 @@ public Table getTable() return table; } + public HiveIdentity getIdentity() + { + return identity; + } + public PrincipalPrivileges getPrincipalPrivileges() { checkState(principalPrivileges.isPresent()); @@ -2129,14 +2163,16 @@ public String toString() private static class PartitionAndMore { + private final HiveIdentity identity; private final Partition partition; private final Path currentLocation; private final Optional> fileNames; private final PartitionStatistics statistics; private final PartitionStatistics statisticsUpdate; - public PartitionAndMore(Partition partition, Path currentLocation, Optional> fileNames, PartitionStatistics statistics, PartitionStatistics statisticsUpdate) + public PartitionAndMore(HiveIdentity identity, Partition partition, Path currentLocation, Optional> fileNames, PartitionStatistics statistics, PartitionStatistics statisticsUpdate) { + this.identity = requireNonNull(identity, "identity is null"); this.partition = requireNonNull(partition, "partition is null"); this.currentLocation = requireNonNull(currentLocation, "currentLocation is null"); this.fileNames = requireNonNull(fileNames, "fileNames is null"); @@ -2144,6 +2180,11 @@ public PartitionAndMore(Partition partition, Path currentLocation, Optional existingTable = metastore.getTable(newTable.getDatabaseName(), newTable.getTableName()); + Optional
existingTable = metastore.getTable(identity, newTable.getDatabaseName(), newTable.getTableName()); if (existingTable.isPresent()) { Table table = existingTable.get(); Optional existingTableQueryId = getPrestoQueryId(table); @@ -2473,19 +2524,21 @@ public void undo(HiveMetastore metastore) if (!tableCreated) { return; } - metastore.dropTable(newTable.getDatabaseName(), newTable.getTableName(), false); + metastore.dropTable(identity, newTable.getDatabaseName(), newTable.getTableName(), false); } } private static class AlterTableOperation { + private final HiveIdentity identity; private final Table newTable; private final Table oldTable; private final PrincipalPrivileges principalPrivileges; private boolean undo; - public AlterTableOperation(Table newTable, Table oldTable, PrincipalPrivileges principalPrivileges) + public AlterTableOperation(HiveIdentity identity, Table newTable, Table oldTable, PrincipalPrivileges principalPrivileges) { + this.identity = requireNonNull(identity, "identity is null"); this.newTable = requireNonNull(newTable, "newTable is null"); this.oldTable = requireNonNull(oldTable, "oldTable is null"); this.principalPrivileges = requireNonNull(principalPrivileges, "principalPrivileges is null"); @@ -2504,7 +2557,7 @@ public String getDescription() public void run(HiveMetastore metastore) { undo = true; - metastore.replaceTable(newTable.getDatabaseName(), newTable.getTableName(), newTable, principalPrivileges); + metastore.replaceTable(identity, newTable.getDatabaseName(), newTable.getTableName(), newTable, principalPrivileges); } public void undo(HiveMetastore metastore) @@ -2513,18 +2566,20 @@ public void undo(HiveMetastore metastore) return; } - metastore.replaceTable(oldTable.getDatabaseName(), oldTable.getTableName(), oldTable, principalPrivileges); + metastore.replaceTable(identity, oldTable.getDatabaseName(), oldTable.getTableName(), oldTable, principalPrivileges); } } private static class AlterPartitionOperation { + private final HiveIdentity identity; private final PartitionWithStatistics newPartition; private final PartitionWithStatistics oldPartition; private boolean undo; - public AlterPartitionOperation(PartitionWithStatistics newPartition, PartitionWithStatistics oldPartition) + public AlterPartitionOperation(HiveIdentity identity, PartitionWithStatistics newPartition, PartitionWithStatistics oldPartition) { + this.identity = requireNonNull(identity, "identity is null"); this.newPartition = requireNonNull(newPartition, "newPartition is null"); this.oldPartition = requireNonNull(oldPartition, "oldPartition is null"); checkArgument(newPartition.getPartition().getDatabaseName().equals(oldPartition.getPartition().getDatabaseName())); @@ -2544,7 +2599,7 @@ public String getDescription() public void run(HiveMetastore metastore) { undo = true; - metastore.alterPartition(newPartition.getPartition().getDatabaseName(), newPartition.getPartition().getTableName(), newPartition); + metastore.alterPartition(identity, newPartition.getPartition().getDatabaseName(), newPartition.getPartition().getTableName(), newPartition); } public void undo(HiveMetastore metastore) @@ -2552,12 +2607,13 @@ public void undo(HiveMetastore metastore) if (!undo) { return; } - metastore.alterPartition(oldPartition.getPartition().getDatabaseName(), oldPartition.getPartition().getTableName(), oldPartition); + metastore.alterPartition(identity, oldPartition.getPartition().getDatabaseName(), oldPartition.getPartition().getTableName(), oldPartition); } } private static class UpdateStatisticsOperation { + private final HiveIdentity identity; private final SchemaTableName tableName; private final Optional partitionName; private final PartitionStatistics statistics; @@ -2565,8 +2621,9 @@ private static class UpdateStatisticsOperation private boolean done; - public UpdateStatisticsOperation(SchemaTableName tableName, Optional partitionName, PartitionStatistics statistics, boolean merge) + public UpdateStatisticsOperation(HiveIdentity identity, SchemaTableName tableName, Optional partitionName, PartitionStatistics statistics, boolean merge) { + this.identity = requireNonNull(identity, "identity is null"); this.tableName = requireNonNull(tableName, "tableName is null"); this.partitionName = requireNonNull(partitionName, "partitionValues is null"); this.statistics = requireNonNull(statistics, "statistics is null"); @@ -2576,10 +2633,10 @@ public UpdateStatisticsOperation(SchemaTableName tableName, Optional par public void run(HiveMetastore metastore) { if (partitionName.isPresent()) { - metastore.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName.get(), this::updateStatistics); + metastore.updatePartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), partitionName.get(), this::updateStatistics); } else { - metastore.updateTableStatistics(tableName.getSchemaName(), tableName.getTableName(), this::updateStatistics); + metastore.updateTableStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), this::updateStatistics); } done = true; } @@ -2590,10 +2647,10 @@ public void undo(HiveMetastore metastore) return; } if (partitionName.isPresent()) { - metastore.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName.get(), this::resetStatistics); + metastore.updatePartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), partitionName.get(), this::resetStatistics); } else { - metastore.updateTableStatistics(tableName.getSchemaName(), tableName.getTableName(), this::resetStatistics); + metastore.updateTableStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), this::resetStatistics); } } @@ -2618,6 +2675,7 @@ private PartitionStatistics resetStatistics(PartitionStatistics currentStatistic private static class PartitionAdder { + private final HiveIdentity identity; private final String schemaName; private final String tableName; private final HiveMetastore metastore; @@ -2625,8 +2683,9 @@ private static class PartitionAdder private final List partitions; private List> createdPartitionValues = new ArrayList<>(); - public PartitionAdder(String schemaName, String tableName, HiveMetastore metastore, int batchSize) + public PartitionAdder(HiveIdentity identity, String schemaName, String tableName, HiveMetastore metastore, int batchSize) { + this.identity = identity; this.schemaName = schemaName; this.tableName = tableName; this.metastore = metastore; @@ -2655,7 +2714,7 @@ public void execute() List> batchedPartitions = Lists.partition(partitions, batchSize); for (List batch : batchedPartitions) { try { - metastore.addPartitions(schemaName, tableName, batch); + metastore.addPartitions(identity, schemaName, tableName, batch); for (PartitionWithStatistics partition : batch) { createdPartitionValues.add(partition.getPartition().getValues()); } @@ -2666,7 +2725,7 @@ public void execute() boolean batchCompletelyAdded = true; for (PartitionWithStatistics partition : batch) { try { - Optional remotePartition = metastore.getPartition(schemaName, tableName, partition.getPartition().getValues()); + Optional remotePartition = metastore.getPartition(identity, schemaName, tableName, partition.getPartition().getValues()); // getPrestoQueryId(partition) is guaranteed to be non-empty. It is asserted in PartitionAdder.addPartition. if (remotePartition.isPresent() && getPrestoQueryId(remotePartition.get()).equals(getPrestoQueryId(partition.getPartition()))) { createdPartitionValues.add(partition.getPartition().getValues()); @@ -2703,7 +2762,7 @@ public List> rollback() List> partitionsFailedToRollback = new ArrayList<>(); for (List createdPartitionValue : createdPartitionValues) { try { - metastore.dropPartition(schemaName, tableName, createdPartitionValue, false); + metastore.dropPartition(identity, schemaName, tableName, createdPartitionValue, false); } catch (PartitionNotFoundException e) { // Maybe some one deleted the partition we added. diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/cache/CachingHiveMetastore.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/cache/CachingHiveMetastore.java index eeb003cb9acb8..7d9f636d160e5 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/cache/CachingHiveMetastore.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/cache/CachingHiveMetastore.java @@ -24,6 +24,7 @@ import io.airlift.units.Duration; import io.prestosql.plugin.hive.HiveType; import io.prestosql.plugin.hive.PartitionStatistics; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Database; import io.prestosql.plugin.hive.metastore.HiveMetastore; import io.prestosql.plugin.hive.metastore.HivePartitionName; @@ -50,6 +51,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; import java.util.Optional; import java.util.OptionalLong; import java.util.Set; @@ -57,6 +59,7 @@ import java.util.concurrent.Executor; import java.util.function.Function; +import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Throwables.throwIfInstanceOf; import static com.google.common.base.Throwables.throwIfUnchecked; @@ -85,15 +88,15 @@ public class CachingHiveMetastore protected final HiveMetastore delegate; private final LoadingCache> databaseCache; private final LoadingCache> databaseNamesCache; - private final LoadingCache> tableCache; + private final LoadingCache, Optional
> tableCache; private final LoadingCache> tableNamesCache; private final LoadingCache> tablesWithParameterCache; - private final LoadingCache tableStatisticsCache; - private final LoadingCache partitionStatisticsCache; + private final LoadingCache, PartitionStatistics> tableStatisticsCache; + private final LoadingCache, PartitionStatistics> partitionStatisticsCache; private final LoadingCache> viewNamesCache; - private final LoadingCache> partitionCache; - private final LoadingCache>> partitionFilterCache; - private final LoadingCache>> partitionNamesCache; + private final LoadingCache, Optional> partitionCache; + private final LoadingCache, Optional>> partitionFilterCache; + private final LoadingCache, Optional>> partitionNamesCache; private final LoadingCache> tablePrivilegesCache; private final LoadingCache> rolesCache; private final LoadingCache> roleGrantsCache; @@ -147,26 +150,26 @@ private CachingHiveMetastore(HiveMetastore delegate, Executor executor, Optional .build(asyncReloading(CacheLoader.from(this::loadTablesMatchingParameter), executor)); tableStatisticsCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) - .build(asyncReloading(new CacheLoader() + .build(asyncReloading(new CacheLoader, PartitionStatistics>() { @Override - public PartitionStatistics load(HiveTableName key) + public PartitionStatistics load(WithIdentity key) { return loadTableColumnStatistics(key); } }, executor)); partitionStatisticsCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) - .build(asyncReloading(new CacheLoader() + .build(asyncReloading(new CacheLoader, PartitionStatistics>() { @Override - public PartitionStatistics load(HivePartitionName key) + public PartitionStatistics load(WithIdentity key) { return loadPartitionColumnStatistics(key); } @Override - public Map loadAll(Iterable keys) + public Map, PartitionStatistics> loadAll(Iterable> keys) { return loadPartitionColumnStatistics(keys); } @@ -185,16 +188,16 @@ public Map loadAll(Iterable>() + .build(asyncReloading(new CacheLoader, Optional>() { @Override - public Optional load(HivePartitionName partitionName) + public Optional load(WithIdentity partitionName) { return loadPartitionByName(partitionName); } @Override - public Map> loadAll(Iterable partitionNames) + public Map, Optional> loadAll(Iterable> partitionNames) { return loadPartitionsByNames(partitionNames); } @@ -273,9 +276,10 @@ private List loadAllDatabases() } @Override - public Optional
getTable(String databaseName, String tableName) + public Optional
getTable(HiveIdentity identity, String databaseName, String tableName) { - return get(tableCache, hiveTableName(databaseName, tableName)); + identity = updateIdentity(identity); + return get(tableCache, new WithIdentity<>(identity, hiveTableName(databaseName, tableName))); } @Override @@ -284,86 +288,89 @@ public Set getSupportedColumnStatistics(Type type) return delegate.getSupportedColumnStatistics(type); } - private Optional
loadTable(HiveTableName hiveTableName) + private Optional
loadTable(WithIdentity hiveTableName) { - return delegate.getTable(hiveTableName.getDatabaseName(), hiveTableName.getTableName()); + return delegate.getTable(hiveTableName.getIdentity(), hiveTableName.key.getDatabaseName(), hiveTableName.key.getTableName()); } @Override - public PartitionStatistics getTableStatistics(String databaseName, String tableName) + public PartitionStatistics getTableStatistics(HiveIdentity identity, String databaseName, String tableName) { - return get(tableStatisticsCache, hiveTableName(databaseName, tableName)); + return get(tableStatisticsCache, new WithIdentity<>(updateIdentity(identity), hiveTableName(databaseName, tableName))); } - private PartitionStatistics loadTableColumnStatistics(HiveTableName hiveTableName) + private PartitionStatistics loadTableColumnStatistics(WithIdentity hiveTableName) { - return delegate.getTableStatistics(hiveTableName.getDatabaseName(), hiveTableName.getTableName()); + return delegate.getTableStatistics(hiveTableName.getIdentity(), hiveTableName.getKey().getDatabaseName(), hiveTableName.getKey().getTableName()); } @Override - public Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames) + public Map getPartitionStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames) { - List partitions = partitionNames.stream() - .map(partitionName -> HivePartitionName.hivePartitionName(databaseName, tableName, partitionName)) + List> partitions = partitionNames.stream() + .map(partitionName -> new WithIdentity<>(updateIdentity(identity), hivePartitionName(databaseName, tableName, partitionName))) .collect(toImmutableList()); - Map statistics = getAll(partitionStatisticsCache, partitions); + Map, PartitionStatistics> statistics = getAll(partitionStatisticsCache, partitions); return statistics.entrySet() .stream() - .collect(toImmutableMap(entry -> entry.getKey().getPartitionName().get(), Entry::getValue)); + .collect(toImmutableMap(entry -> entry.getKey().getKey().getPartitionName().get(), Entry::getValue)); } - private PartitionStatistics loadPartitionColumnStatistics(HivePartitionName partition) + private PartitionStatistics loadPartitionColumnStatistics(WithIdentity partition) { - String partitionName = partition.getPartitionName().get(); + String partitionName = partition.getKey().getPartitionName().get(); Map partitionStatistics = delegate.getPartitionStatistics( - partition.getHiveTableName().getDatabaseName(), - partition.getHiveTableName().getTableName(), + partition.getIdentity(), + partition.getKey().getHiveTableName().getDatabaseName(), + partition.getKey().getHiveTableName().getTableName(), ImmutableSet.of(partitionName)); if (!partitionStatistics.containsKey(partitionName)) { - throw new PrestoException(HIVE_PARTITION_DROPPED_DURING_QUERY, "Statistics result does not contain entry for partition: " + partition.getPartitionName()); + throw new PrestoException(HIVE_PARTITION_DROPPED_DURING_QUERY, "Statistics result does not contain entry for partition: " + partition.getKey().getPartitionName()); } return partitionStatistics.get(partitionName); } - private Map loadPartitionColumnStatistics(Iterable keys) + private Map, PartitionStatistics> loadPartitionColumnStatistics(Iterable> keys) { - SetMultimap tablePartitions = stream(keys) - .collect(toImmutableSetMultimap(HivePartitionName::getHiveTableName, key -> key)); - ImmutableMap.Builder result = ImmutableMap.builder(); + SetMultimap, WithIdentity> tablePartitions = stream(keys) + .collect(toImmutableSetMultimap(value -> new WithIdentity<>(value.getIdentity(), value.getKey().getHiveTableName()), key -> key)); + ImmutableMap.Builder, PartitionStatistics> result = ImmutableMap.builder(); tablePartitions.keySet().forEach(table -> { Set partitionNames = tablePartitions.get(table).stream() - .map(partitionName -> partitionName.getPartitionName().get()) + .map(partitionName -> partitionName.getKey().getPartitionName().get()) .collect(toImmutableSet()); - Map partitionStatistics = delegate.getPartitionStatistics(table.getDatabaseName(), table.getTableName(), partitionNames); + Map partitionStatistics = delegate.getPartitionStatistics(table.getIdentity(), table.getKey().getDatabaseName(), table.getKey().getTableName(), partitionNames); for (String partitionName : partitionNames) { if (!partitionStatistics.containsKey(partitionName)) { throw new PrestoException(HIVE_PARTITION_DROPPED_DURING_QUERY, "Statistics result does not contain entry for partition: " + partitionName); } - result.put(HivePartitionName.hivePartitionName(table, partitionName), partitionStatistics.get(partitionName)); + result.put(new WithIdentity<>(table.getIdentity(), hivePartitionName(table.getKey(), partitionName)), partitionStatistics.get(partitionName)); } }); return result.build(); } @Override - public void updateTableStatistics(String databaseName, String tableName, Function update) + public void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, Function update) { + identity = updateIdentity(identity); try { - delegate.updateTableStatistics(databaseName, tableName, update); + delegate.updateTableStatistics(identity, databaseName, tableName, update); } finally { - tableStatisticsCache.invalidate(hiveTableName(databaseName, tableName)); + tableStatisticsCache.invalidate(new WithIdentity<>(identity, hiveTableName(databaseName, tableName))); } } @Override - public void updatePartitionStatistics(String databaseName, String tableName, String partitionName, Function update) + public void updatePartitionStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, Function update) { + identity = updateIdentity(identity); try { - delegate.updatePartitionStatistics(databaseName, tableName, partitionName, update); + delegate.updatePartitionStatistics(identity, databaseName, tableName, partitionName, update); } finally { - partitionStatisticsCache.invalidate(HivePartitionName.hivePartitionName(databaseName, tableName, partitionName)); + partitionStatisticsCache.invalidate(new WithIdentity<>(identity, hivePartitionName(databaseName, tableName, partitionName))); } } @@ -402,10 +409,11 @@ private List loadAllViews(String databaseName) } @Override - public void createDatabase(Database database) + public void createDatabase(HiveIdentity identity, Database database) { + identity = updateIdentity(identity); try { - delegate.createDatabase(database); + delegate.createDatabase(identity, database); } finally { invalidateDatabase(database.getDatabaseName()); @@ -413,10 +421,11 @@ public void createDatabase(Database database) } @Override - public void dropDatabase(String databaseName) + public void dropDatabase(HiveIdentity identity, String databaseName) { + identity = updateIdentity(identity); try { - delegate.dropDatabase(databaseName); + delegate.dropDatabase(identity, databaseName); } finally { invalidateDatabase(databaseName); @@ -424,10 +433,11 @@ public void dropDatabase(String databaseName) } @Override - public void renameDatabase(String databaseName, String newDatabaseName) + public void renameDatabase(HiveIdentity identity, String databaseName, String newDatabaseName) { + identity = updateIdentity(identity); try { - delegate.renameDatabase(databaseName, newDatabaseName); + delegate.renameDatabase(identity, databaseName, newDatabaseName); } finally { invalidateDatabase(databaseName); @@ -442,10 +452,11 @@ protected void invalidateDatabase(String databaseName) } @Override - public void createTable(Table table, PrincipalPrivileges principalPrivileges) + public void createTable(HiveIdentity identity, Table table, PrincipalPrivileges principalPrivileges) { + identity = updateIdentity(identity); try { - delegate.createTable(table, principalPrivileges); + delegate.createTable(identity, table, principalPrivileges); } finally { invalidateTable(table.getDatabaseName(), table.getTableName()); @@ -453,10 +464,11 @@ public void createTable(Table table, PrincipalPrivileges principalPrivileges) } @Override - public void dropTable(String databaseName, String tableName, boolean deleteData) + public void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) { + identity = updateIdentity(identity); try { - delegate.dropTable(databaseName, tableName, deleteData); + delegate.dropTable(identity, databaseName, tableName, deleteData); } finally { invalidateTable(databaseName, tableName); @@ -464,10 +476,11 @@ public void dropTable(String databaseName, String tableName, boolean deleteData) } @Override - public void replaceTable(String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) + public void replaceTable(HiveIdentity identity, String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) { + identity = updateIdentity(identity); try { - delegate.replaceTable(databaseName, tableName, newTable, principalPrivileges); + delegate.replaceTable(identity, databaseName, tableName, newTable, principalPrivileges); } finally { invalidateTable(databaseName, tableName); @@ -476,10 +489,11 @@ public void replaceTable(String databaseName, String tableName, Table newTable, } @Override - public void renameTable(String databaseName, String tableName, String newDatabaseName, String newTableName) + public void renameTable(HiveIdentity identity, String databaseName, String tableName, String newDatabaseName, String newTableName) { + identity = updateIdentity(identity); try { - delegate.renameTable(databaseName, tableName, newDatabaseName, newTableName); + delegate.renameTable(identity, databaseName, tableName, newDatabaseName, newTableName); } finally { invalidateTable(databaseName, tableName); @@ -488,10 +502,11 @@ public void renameTable(String databaseName, String tableName, String newDatabas } @Override - public void commentTable(String databaseName, String tableName, Optional comment) + public void commentTable(HiveIdentity identity, String databaseName, String tableName, Optional comment) { + identity = updateIdentity(identity); try { - delegate.commentTable(databaseName, tableName, comment); + delegate.commentTable(identity, databaseName, tableName, comment); } finally { invalidateTable(databaseName, tableName); @@ -499,10 +514,11 @@ public void commentTable(String databaseName, String tableName, Optional } @Override - public void addColumn(String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) + public void addColumn(HiveIdentity identity, String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) { + identity = updateIdentity(identity); try { - delegate.addColumn(databaseName, tableName, columnName, columnType, columnComment); + delegate.addColumn(identity, databaseName, tableName, columnName, columnType, columnComment); } finally { invalidateTable(databaseName, tableName); @@ -510,10 +526,11 @@ public void addColumn(String databaseName, String tableName, String columnName, } @Override - public void renameColumn(String databaseName, String tableName, String oldColumnName, String newColumnName) + public void renameColumn(HiveIdentity identity, String databaseName, String tableName, String oldColumnName, String newColumnName) { + identity = updateIdentity(identity); try { - delegate.renameColumn(databaseName, tableName, oldColumnName, newColumnName); + delegate.renameColumn(identity, databaseName, tableName, oldColumnName, newColumnName); } finally { invalidateTable(databaseName, tableName); @@ -521,10 +538,11 @@ public void renameColumn(String databaseName, String tableName, String oldColumn } @Override - public void dropColumn(String databaseName, String tableName, String columnName) + public void dropColumn(HiveIdentity identity, String databaseName, String tableName, String columnName) { + identity = updateIdentity(identity); try { - delegate.dropColumn(databaseName, tableName, columnName); + delegate.dropColumn(identity, databaseName, tableName, columnName); } finally { invalidateTable(databaseName, tableName); @@ -533,99 +551,120 @@ public void dropColumn(String databaseName, String tableName, String columnName) protected void invalidateTable(String databaseName, String tableName) { - tableCache.invalidate(hiveTableName(databaseName, tableName)); + invalidateTableCache(databaseName, tableName); tableNamesCache.invalidate(databaseName); viewNamesCache.invalidate(databaseName); tablePrivilegesCache.asMap().keySet().stream() .filter(userTableKey -> userTableKey.matches(databaseName, tableName)) .forEach(tablePrivilegesCache::invalidate); - tableStatisticsCache.invalidate(hiveTableName(databaseName, tableName)); + invalidateTableStatisticsCache(databaseName, tableName); invalidatePartitionCache(databaseName, tableName); } + private void invalidateTableCache(String databaseName, String tableName) + { + tableCache.asMap().keySet().stream() + .filter(table -> table.getKey().getDatabaseName().equals(databaseName) && table.getKey().getTableName().equals(tableName)) + .forEach(tableCache::invalidate); + } + + private void invalidateTableStatisticsCache(String databaseName, String tableName) + { + tableStatisticsCache.asMap().keySet().stream() + .filter(table -> table.getKey().getDatabaseName().equals(databaseName) && table.getKey().getTableName().equals(tableName)) + .forEach(tableCache::invalidate); + } + @Override - public Optional getPartition(String databaseName, String tableName, List partitionValues) + public Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues) { - HivePartitionName name = hivePartitionName(databaseName, tableName, partitionValues); + identity = updateIdentity(identity); + WithIdentity name = new WithIdentity<>(identity, hivePartitionName(databaseName, tableName, partitionValues)); return get(partitionCache, name); } @Override - public Optional> getPartitionNames(String databaseName, String tableName) + public Optional> getPartitionNames(HiveIdentity identity, String databaseName, String tableName) { - return get(partitionNamesCache, hiveTableName(databaseName, tableName)); + identity = updateIdentity(identity); + return get(partitionNamesCache, new WithIdentity<>(identity, hiveTableName(databaseName, tableName))); } - private Optional> loadPartitionNames(HiveTableName hiveTableName) + private Optional> loadPartitionNames(WithIdentity hiveTableName) { - return delegate.getPartitionNames(hiveTableName.getDatabaseName(), hiveTableName.getTableName()); + return delegate.getPartitionNames(hiveTableName.getIdentity(), hiveTableName.getKey().getDatabaseName(), hiveTableName.getKey().getTableName()); } @Override - public Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) + public Optional> getPartitionNamesByParts(HiveIdentity identity, String databaseName, String tableName, List parts) { - return get(partitionFilterCache, partitionFilter(databaseName, tableName, parts)); + identity = updateIdentity(identity); + return get(partitionFilterCache, new WithIdentity<>(identity, partitionFilter(databaseName, tableName, parts))); } - private Optional> loadPartitionNamesByParts(PartitionFilter partitionFilter) + private Optional> loadPartitionNamesByParts(WithIdentity partitionFilter) { return delegate.getPartitionNamesByParts( - partitionFilter.getHiveTableName().getDatabaseName(), - partitionFilter.getHiveTableName().getTableName(), - partitionFilter.getParts()); + partitionFilter.getIdentity(), + partitionFilter.getKey().getHiveTableName().getDatabaseName(), + partitionFilter.getKey().getHiveTableName().getTableName(), + partitionFilter.getKey().getParts()); } @Override - public Map> getPartitionsByNames(String databaseName, String tableName, List partitionNames) + public Map> getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames) { - Iterable names = transform(partitionNames, name -> HivePartitionName.hivePartitionName(databaseName, tableName, name)); + Iterable> names = transform(partitionNames, name -> new WithIdentity<>(updateIdentity(identity), hivePartitionName(databaseName, tableName, name))); - Map> all = getAll(partitionCache, names); + Map, Optional> all = getAll(partitionCache, names); ImmutableMap.Builder> partitionsByName = ImmutableMap.builder(); - for (Entry> entry : all.entrySet()) { - partitionsByName.put(entry.getKey().getPartitionName().get(), entry.getValue()); + for (Entry, Optional> entry : all.entrySet()) { + partitionsByName.put(entry.getKey().getKey().getPartitionName().get(), entry.getValue()); } return partitionsByName.build(); } - private Optional loadPartitionByName(HivePartitionName partitionName) + private Optional loadPartitionByName(WithIdentity partitionName) { return delegate.getPartition( - partitionName.getHiveTableName().getDatabaseName(), - partitionName.getHiveTableName().getTableName(), - partitionName.getPartitionValues()); + partitionName.getIdentity(), + partitionName.getKey().getHiveTableName().getDatabaseName(), + partitionName.getKey().getHiveTableName().getTableName(), + partitionName.getKey().getPartitionValues()); } - private Map> loadPartitionsByNames(Iterable partitionNames) + private Map, Optional> loadPartitionsByNames(Iterable> partitionNames) { requireNonNull(partitionNames, "partitionNames is null"); checkArgument(!Iterables.isEmpty(partitionNames), "partitionNames is empty"); - HivePartitionName firstPartition = Iterables.get(partitionNames, 0); + WithIdentity firstPartition = Iterables.get(partitionNames, 0); - HiveTableName hiveTableName = firstPartition.getHiveTableName(); + HiveTableName hiveTableName = firstPartition.getKey().getHiveTableName(); + HiveIdentity identity = updateIdentity(firstPartition.getIdentity()); String databaseName = hiveTableName.getDatabaseName(); String tableName = hiveTableName.getTableName(); List partitionsToFetch = new ArrayList<>(); - for (HivePartitionName partitionName : partitionNames) { - checkArgument(partitionName.getHiveTableName().equals(hiveTableName), "Expected table name %s but got %s", hiveTableName, partitionName.getHiveTableName()); - partitionsToFetch.add(partitionName.getPartitionName().get()); + for (WithIdentity partitionName : partitionNames) { + checkArgument(partitionName.getKey().getHiveTableName().equals(hiveTableName), "Expected table name %s but got %s", hiveTableName, partitionName.getKey().getHiveTableName()); + partitionsToFetch.add(partitionName.getKey().getPartitionName().get()); } - ImmutableMap.Builder> partitions = ImmutableMap.builder(); - Map> partitionsByNames = delegate.getPartitionsByNames(databaseName, tableName, partitionsToFetch); + ImmutableMap.Builder, Optional> partitions = ImmutableMap.builder(); + Map> partitionsByNames = delegate.getPartitionsByNames(identity, databaseName, tableName, partitionsToFetch); for (Entry> entry : partitionsByNames.entrySet()) { - partitions.put(HivePartitionName.hivePartitionName(hiveTableName, entry.getKey()), entry.getValue()); + partitions.put(new WithIdentity<>(identity, hivePartitionName(hiveTableName, entry.getKey())), entry.getValue()); } return partitions.build(); } @Override - public void addPartitions(String databaseName, String tableName, List partitions) + public void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitions) { + identity = updateIdentity(identity); try { - delegate.addPartitions(databaseName, tableName, partitions); + delegate.addPartitions(identity, databaseName, tableName, partitions); } finally { // todo do we need to invalidate all partitions? @@ -634,10 +673,11 @@ public void addPartitions(String databaseName, String tableName, List parts, boolean deleteData) + public void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData) { + identity = updateIdentity(identity); try { - delegate.dropPartition(databaseName, tableName, parts, deleteData); + delegate.dropPartition(identity, databaseName, tableName, parts, deleteData); } finally { invalidatePartitionCache(databaseName, tableName); @@ -645,10 +685,11 @@ public void dropPartition(String databaseName, String tableName, List pa } @Override - public void alterPartition(String databaseName, String tableName, PartitionWithStatistics partition) + public void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partition) { + identity = updateIdentity(identity); try { - delegate.alterPartition(databaseName, tableName, partition); + delegate.alterPartition(identity, databaseName, tableName, partition); } finally { invalidatePartitionCache(databaseName, tableName); @@ -725,15 +766,17 @@ private Set loadRoleGrants(HivePrincipal principal) private void invalidatePartitionCache(String databaseName, String tableName) { HiveTableName hiveTableName = hiveTableName(databaseName, tableName); - partitionNamesCache.invalidate(hiveTableName); + partitionNamesCache.asMap().keySet().stream() + .filter(partitionName -> partitionName.getKey().equals(hiveTableName)) + .forEach(partitionNamesCache::invalidate); partitionCache.asMap().keySet().stream() - .filter(partitionName -> partitionName.getHiveTableName().equals(hiveTableName)) + .filter(partitionName -> partitionName.getKey().getHiveTableName().equals(hiveTableName)) .forEach(partitionCache::invalidate); partitionFilterCache.asMap().keySet().stream() - .filter(partitionFilter -> partitionFilter.getHiveTableName().equals(hiveTableName)) + .filter(partitionFilter -> partitionFilter.getKey().getHiveTableName().equals(hiveTableName)) .forEach(partitionFilterCache::invalidate); partitionStatisticsCache.asMap().keySet().stream() - .filter(partitionFilter -> partitionFilter.getHiveTableName().equals(hiveTableName)) + .filter(partitionFilter -> partitionFilter.getKey().getHiveTableName().equals(hiveTableName)) .forEach(partitionStatisticsCache::invalidate); } @@ -765,6 +808,12 @@ public Set listTablePrivileges(String databaseName, String ta return get(tablePrivilegesCache, new UserTableKey(principal, databaseName, tableName, tableOwner)); } + @Override + public boolean isImpersonationEnabled() + { + return delegate.isImpersonationEnabled(); + } + private Set loadTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal principal) { return delegate.listTablePrivileges(databaseName, tableName, tableOwner, principal); @@ -782,4 +831,61 @@ private static CacheBuilder newCacheBuilder(OptionalLong expires cacheBuilder = cacheBuilder.maximumSize(maximumSize); return cacheBuilder; } + + private static class WithIdentity + { + private final HiveIdentity identity; + private final T key; + + public WithIdentity(HiveIdentity identity, T key) + { + this.identity = requireNonNull(identity, "identity is null"); + this.key = requireNonNull(key, "key is null"); + } + + public HiveIdentity getIdentity() + { + return identity; + } + + public T getKey() + { + return key; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WithIdentity other = (WithIdentity) o; + return Objects.equals(identity, other.identity) && + Objects.equals(key, other.key); + } + + @Override + public int hashCode() + { + return Objects.hash(identity, key); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("identity", identity) + .add("key", key) + .toString(); + } + } + + private HiveIdentity updateIdentity(HiveIdentity identity) + { + // remove identity if not doing impersonation + return delegate.isImpersonationEnabled() ? identity : HiveIdentity.none(); + } } 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 6de53accd57ec..d675e371fbbf0 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 @@ -30,6 +30,7 @@ import io.prestosql.plugin.hive.PartitionStatistics; import io.prestosql.plugin.hive.SchemaAlreadyExistsException; import io.prestosql.plugin.hive.TableAlreadyExistsException; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.authentication.NoHdfsAuthentication; import io.prestosql.plugin.hive.metastore.Column; import io.prestosql.plugin.hive.metastore.Database; @@ -162,7 +163,7 @@ public FileHiveMetastore(HdfsEnvironment hdfsEnvironment, String catalogDirector } @Override - public synchronized void createDatabase(Database database) + public synchronized void createDatabase(HiveIdentity identity, Database database) { requireNonNull(database, "database is null"); @@ -177,7 +178,7 @@ public synchronized void createDatabase(Database database) } @Override - public synchronized void dropDatabase(String databaseName) + public synchronized void dropDatabase(HiveIdentity identity, String databaseName) { requireNonNull(databaseName, "databaseName is null"); @@ -190,7 +191,7 @@ public synchronized void dropDatabase(String databaseName) } @Override - public synchronized void renameDatabase(String databaseName, String newDatabaseName) + public synchronized void renameDatabase(HiveIdentity identity, String databaseName, String newDatabaseName) { requireNonNull(databaseName, "databaseName is null"); requireNonNull(newDatabaseName, "newDatabaseName is null"); @@ -241,7 +242,7 @@ public synchronized List getAllDatabases() } @Override - public synchronized void createTable(Table table, PrincipalPrivileges principalPrivileges) + public synchronized void createTable(HiveIdentity identity, Table table, PrincipalPrivileges principalPrivileges) { verifyTableNotExists(table.getDatabaseName(), table.getTableName()); @@ -286,7 +287,12 @@ else if (table.getTableType().equals(EXTERNAL_TABLE.name())) { } @Override - public synchronized Optional
getTable(String databaseName, String tableName) + public synchronized Optional
getTable(HiveIdentity identity, String databaseName, String tableName) + { + return getTable(databaseName, tableName); + } + + private Optional
getTable(String databaseName, String tableName) { requireNonNull(databaseName, "databaseName is null"); requireNonNull(tableName, "tableName is null"); @@ -303,7 +309,7 @@ public Set getSupportedColumnStatistics(Type type) } @Override - public synchronized PartitionStatistics getTableStatistics(String databaseName, String tableName) + public synchronized PartitionStatistics getTableStatistics(HiveIdentity identity, String databaseName, String tableName) { Path tableMetadataDirectory = getTableMetadataDirectory(databaseName, tableName); TableMetadata tableMetadata = readSchemaFile("table", tableMetadataDirectory, tableCodec) @@ -314,7 +320,7 @@ public synchronized PartitionStatistics getTableStatistics(String databaseName, } @Override - public synchronized Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames) + public synchronized Map getPartitionStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames) { Table table = getRequiredTable(databaseName, tableName); ImmutableMap.Builder statistics = ImmutableMap.builder(); @@ -343,9 +349,9 @@ private void verifyTableNotExists(String newDatabaseName, String newTableName) } @Override - public synchronized void updateTableStatistics(String databaseName, String tableName, Function update) + public synchronized void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, Function update) { - PartitionStatistics originalStatistics = getTableStatistics(databaseName, tableName); + PartitionStatistics originalStatistics = getTableStatistics(identity, databaseName, tableName); PartitionStatistics updatedStatistics = update.apply(originalStatistics); Path tableMetadataDirectory = getTableMetadataDirectory(databaseName, tableName); @@ -360,9 +366,9 @@ public synchronized void updateTableStatistics(String databaseName, String table } @Override - public synchronized void updatePartitionStatistics(String databaseName, String tableName, String partitionName, Function update) + public synchronized void updatePartitionStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, Function update) { - PartitionStatistics originalStatistics = getPartitionStatistics(databaseName, tableName, ImmutableSet.of(partitionName)).get(partitionName); + PartitionStatistics originalStatistics = getPartitionStatistics(identity, databaseName, tableName, ImmutableSet.of(partitionName)).get(partitionName); if (originalStatistics == null) { throw new PrestoException(HIVE_PARTITION_DROPPED_DURING_QUERY, "Statistics result does not contain entry for partition: " + partitionName); } @@ -422,7 +428,7 @@ public synchronized List getAllViews(String databaseName) } @Override - public synchronized void dropTable(String databaseName, String tableName, boolean deleteData) + public synchronized void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) { requireNonNull(databaseName, "databaseName is null"); requireNonNull(tableName, "tableName is null"); @@ -443,7 +449,7 @@ public synchronized void dropTable(String databaseName, String tableName, boolea } @Override - public synchronized void replaceTable(String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) + public synchronized void replaceTable(HiveIdentity identity, String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) { Table table = getRequiredTable(databaseName, tableName); if (!table.getDatabaseName().equals(databaseName) || !table.getTableName().equals(tableName)) { @@ -465,7 +471,7 @@ public synchronized void replaceTable(String databaseName, String tableName, Tab } @Override - public synchronized void renameTable(String databaseName, String tableName, String newDatabaseName, String newTableName) + public synchronized void renameTable(HiveIdentity identity, String databaseName, String tableName, String newDatabaseName, String newTableName) { requireNonNull(databaseName, "databaseName is null"); requireNonNull(tableName, "tableName is null"); @@ -496,7 +502,7 @@ public synchronized void renameTable(String databaseName, String tableName, Stri } @Override - public synchronized void commentTable(String databaseName, String tableName, Optional comment) + public synchronized void commentTable(HiveIdentity identity, String databaseName, String tableName, Optional comment) { alterTable(databaseName, tableName, oldTable -> { Map parameters = oldTable.getParameters().entrySet().stream() @@ -511,7 +517,7 @@ public synchronized void commentTable(String databaseName, String tableName, Opt } @Override - public synchronized void addColumn(String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) + public synchronized void addColumn(HiveIdentity identity, String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) { alterTable(databaseName, tableName, oldTable -> { if (oldTable.getColumn(columnName).isPresent()) { @@ -526,7 +532,7 @@ public synchronized void addColumn(String databaseName, String tableName, String } @Override - public synchronized void renameColumn(String databaseName, String tableName, String oldColumnName, String newColumnName) + public synchronized void renameColumn(HiveIdentity identity, String databaseName, String tableName, String oldColumnName, String newColumnName) { alterTable(databaseName, tableName, oldTable -> { if (oldTable.getColumn(newColumnName).isPresent()) { @@ -557,10 +563,10 @@ public synchronized void renameColumn(String databaseName, String tableName, Str } @Override - public synchronized void dropColumn(String databaseName, String tableName, String columnName) + public synchronized void dropColumn(HiveIdentity identity, String databaseName, String tableName, String columnName) { alterTable(databaseName, tableName, oldTable -> { - verifyCanDropColumn(this, databaseName, tableName, columnName); + verifyCanDropColumn(this, identity, databaseName, tableName, columnName); if (!oldTable.getColumn(columnName).isPresent()) { SchemaTableName name = new SchemaTableName(databaseName, tableName); throw new ColumnNotFoundException(name, columnName); @@ -595,7 +601,7 @@ private void alterTable(String databaseName, String tableName, Function partitions) + public synchronized void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitions) { requireNonNull(databaseName, "databaseName is null"); requireNonNull(tableName, "tableName is null"); @@ -678,13 +684,13 @@ else if (table.getTableType().equals(EXTERNAL_TABLE.name())) { } @Override - public synchronized void dropPartition(String databaseName, String tableName, List partitionValues, boolean deleteData) + public synchronized void dropPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues, boolean deleteData) { requireNonNull(databaseName, "databaseName is null"); requireNonNull(tableName, "tableName is null"); requireNonNull(partitionValues, "partitionValues is null"); - Optional
tableReference = getTable(databaseName, tableName); + Optional
tableReference = getTable(identity, databaseName, tableName); if (!tableReference.isPresent()) { return; } @@ -700,7 +706,7 @@ public synchronized void dropPartition(String databaseName, String tableName, Li } @Override - public synchronized void alterPartition(String databaseName, String tableName, PartitionWithStatistics partitionWithStatistics) + public synchronized void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partitionWithStatistics) { Table table = getRequiredTable(databaseName, tableName); @@ -855,12 +861,13 @@ private void writeRoleGrantsFile(Set roleGrants) } @Override - public synchronized Optional> getPartitionNames(String databaseName, String tableName) + public synchronized Optional> getPartitionNames(HiveIdentity identity, String databaseName, String tableName) { + requireNonNull(identity, "identity is null"); requireNonNull(databaseName, "databaseName is null"); requireNonNull(tableName, "tableName is null"); - Optional
tableReference = getTable(databaseName, tableName); + Optional
tableReference = getTable(identity, databaseName, tableName); if (!tableReference.isPresent()) { return Optional.empty(); } @@ -917,13 +924,13 @@ private List> listPartitions(Path director, List part } @Override - public synchronized Optional getPartition(String databaseName, String tableName, List partitionValues) + public synchronized Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues) { requireNonNull(databaseName, "databaseName is null"); requireNonNull(tableName, "tableName is null"); requireNonNull(partitionValues, "partitionValues is null"); - Optional
tableReference = getTable(databaseName, tableName); + Optional
tableReference = getTable(identity, databaseName, tableName); if (!tableReference.isPresent()) { return Optional.empty(); } @@ -935,10 +942,10 @@ public synchronized Optional getPartition(String databaseName, String } @Override - public synchronized Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) + public synchronized Optional> getPartitionNamesByParts(HiveIdentity identity, String databaseName, String tableName, List parts) { // todo this should be more efficient by selectively walking the directory tree - return getPartitionNames(databaseName, tableName).map(partitionNames -> partitionNames.stream() + return getPartitionNames(identity, databaseName, tableName).map(partitionNames -> partitionNames.stream() .filter(partitionName -> partitionMatches(partitionName, parts)) .collect(toList())); } @@ -959,12 +966,12 @@ private static boolean partitionMatches(String partitionName, List parts } @Override - public synchronized Map> getPartitionsByNames(String databaseName, String tableName, List partitionNames) + public synchronized Map> getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames) { ImmutableMap.Builder> builder = ImmutableMap.builder(); for (String partitionName : partitionNames) { List partitionValues = toPartitionValues(partitionName); - builder.put(partitionName, getPartition(databaseName, tableName, partitionValues)); + builder.put(partitionName, getPartition(identity, databaseName, tableName, partitionValues)); } return builder.build(); } @@ -1000,6 +1007,12 @@ public synchronized void revokeTablePrivileges(String databaseName, String table setTablePrivileges(grantee, databaseName, tableName, currentPrivileges); } + @Override + public boolean isImpersonationEnabled() + { + return false; + } + private synchronized void setTablePrivileges( HivePrincipal grantee, String 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 6c6b49fdb464e..4e82c2c70bebf 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 @@ -69,6 +69,7 @@ import io.prestosql.plugin.hive.PartitionStatistics; import io.prestosql.plugin.hive.SchemaAlreadyExistsException; import io.prestosql.plugin.hive.TableAlreadyExistsException; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Column; import io.prestosql.plugin.hive.metastore.Database; import io.prestosql.plugin.hive.metastore.HiveMetastore; @@ -244,7 +245,7 @@ public List getAllDatabases() } @Override - public Optional
getTable(String databaseName, String tableName) + public Optional
getTable(HiveIdentity identity, String databaseName, String tableName) { try { GetTableResult result = glueClient.getTable(new GetTableRequest() @@ -267,25 +268,25 @@ public Set getSupportedColumnStatistics(Type type) return ImmutableSet.of(); } - private Table getTableOrElseThrow(String databaseName, String tableName) + private Table getTableOrElseThrow(HiveIdentity identity, String databaseName, String tableName) { - return getTable(databaseName, tableName) + return getTable(identity, databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); } @Override - public PartitionStatistics getTableStatistics(String databaseName, String tableName) + public PartitionStatistics getTableStatistics(HiveIdentity identity, String databaseName, String tableName) { - Table table = getTable(databaseName, tableName) + Table table = getTable(identity, databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); return new PartitionStatistics(getHiveBasicStatistics(table.getParameters()), ImmutableMap.of()); } @Override - public Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames) + public Map getPartitionStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames) { ImmutableMap.Builder result = ImmutableMap.builder(); - getPartitionsByNames(databaseName, tableName, ImmutableList.copyOf(partitionNames)).forEach((partitionName, optionalPartition) -> { + getPartitionsByNames(identity, databaseName, tableName, ImmutableList.copyOf(partitionNames)).forEach((partitionName, optionalPartition) -> { Partition partition = optionalPartition.orElseThrow(() -> new PartitionNotFoundException(new SchemaTableName(databaseName, tableName), toPartitionValues(partitionName))); PartitionStatistics partitionStatistics = new PartitionStatistics(getHiveBasicStatistics(partition.getParameters()), ImmutableMap.of()); @@ -295,15 +296,15 @@ public Map getPartitionStatistics(String databaseNa } @Override - public void updateTableStatistics(String databaseName, String tableName, Function update) + public void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, Function update) { - PartitionStatistics currentStatistics = getTableStatistics(databaseName, tableName); + PartitionStatistics currentStatistics = getTableStatistics(identity, databaseName, tableName); PartitionStatistics updatedStatistics = update.apply(currentStatistics); if (!updatedStatistics.getColumnStatistics().isEmpty()) { throw new PrestoException(NOT_SUPPORTED, "Glue metastore does not support column level statistics"); } - Table table = getTableOrElseThrow(databaseName, tableName); + Table table = getTableOrElseThrow(identity, databaseName, tableName); try { TableInput tableInput = GlueInputConverter.convertTable(table); @@ -322,9 +323,9 @@ public void updateTableStatistics(String databaseName, String tableName, Functio } @Override - public void updatePartitionStatistics(String databaseName, String tableName, String partitionName, Function update) + public void updatePartitionStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, Function update) { - PartitionStatistics currentStatistics = getPartitionStatistics(databaseName, tableName, ImmutableSet.of(partitionName)).get(partitionName); + PartitionStatistics currentStatistics = getPartitionStatistics(identity, databaseName, tableName, ImmutableSet.of(partitionName)).get(partitionName); if (currentStatistics == null) { throw new PrestoException(HIVE_PARTITION_DROPPED_DURING_QUERY, "Statistics result does not contain entry for partition: " + partitionName); } @@ -334,7 +335,7 @@ public void updatePartitionStatistics(String databaseName, String tableName, Str } List partitionValues = toPartitionValues(partitionName); - Partition partition = getPartition(databaseName, tableName, partitionValues) + Partition partition = getPartition(identity, databaseName, tableName, partitionValues) .orElseThrow(() -> new PartitionNotFoundException(new SchemaTableName(databaseName, tableName), partitionValues)); try { PartitionInput partitionInput = GlueInputConverter.convertPartition(partition); @@ -420,7 +421,7 @@ public List getAllViews(String databaseName) } @Override - public void createDatabase(Database database) + public void createDatabase(HiveIdentity identity, Database database) { if (!database.getLocation().isPresent() && defaultDir.isPresent()) { String databaseLocation = new Path(defaultDir.get(), database.getDatabaseName()).toString(); @@ -446,7 +447,7 @@ public void createDatabase(Database database) } @Override - public void dropDatabase(String databaseName) + public void dropDatabase(HiveIdentity identity, String databaseName) { try { glueClient.deleteDatabase(new DeleteDatabaseRequest().withCatalogId(catalogId).withName(databaseName)); @@ -460,7 +461,7 @@ public void dropDatabase(String databaseName) } @Override - public void renameDatabase(String databaseName, String newDatabaseName) + public void renameDatabase(HiveIdentity identity, String databaseName, String newDatabaseName) { try { Database database = getDatabase(databaseName).orElseThrow(() -> new SchemaNotFoundException(databaseName)); @@ -476,7 +477,7 @@ public void renameDatabase(String databaseName, String newDatabaseName) } @Override - public void createTable(Table table, PrincipalPrivileges principalPrivileges) + public void createTable(HiveIdentity identity, Table table, PrincipalPrivileges principalPrivileges) { try { TableInput input = GlueInputConverter.convertTable(table); @@ -497,9 +498,9 @@ public void createTable(Table table, PrincipalPrivileges principalPrivileges) } @Override - public void dropTable(String databaseName, String tableName, boolean deleteData) + public void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) { - Table table = getTableOrElseThrow(databaseName, tableName); + Table table = getTableOrElseThrow(identity, databaseName, tableName); try { glueClient.deleteTable(new DeleteTableRequest() @@ -534,7 +535,7 @@ private static void deleteDir(HdfsContext context, HdfsEnvironment hdfsEnvironme } @Override - public void replaceTable(String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) + public void replaceTable(HiveIdentity identity, String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) { try { TableInput newTableInput = GlueInputConverter.convertTable(newTable); @@ -552,31 +553,31 @@ public void replaceTable(String databaseName, String tableName, Table newTable, } @Override - public void renameTable(String databaseName, String tableName, String newDatabaseName, String newTableName) + public void renameTable(HiveIdentity identity, String databaseName, String tableName, String newDatabaseName, String newTableName) { throw new PrestoException(NOT_SUPPORTED, "Table rename is not yet supported by Glue service"); } @Override - public void commentTable(String databaseName, String tableName, Optional comment) + public void commentTable(HiveIdentity identity, String databaseName, String tableName, Optional comment) { throw new PrestoException(NOT_SUPPORTED, "Table comment is not yet supported by Glue service"); } @Override - public void addColumn(String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) + public void addColumn(HiveIdentity identity, String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) { - Table oldTable = getTableOrElseThrow(databaseName, tableName); + Table oldTable = getTableOrElseThrow(identity, databaseName, tableName); Table newTable = Table.builder(oldTable) .addDataColumn(new Column(columnName, columnType, Optional.ofNullable(columnComment))) .build(); - replaceTable(databaseName, tableName, newTable, null); + replaceTable(identity, databaseName, tableName, newTable, null); } @Override - public void renameColumn(String databaseName, String tableName, String oldColumnName, String newColumnName) + public void renameColumn(HiveIdentity identity, String databaseName, String tableName, String oldColumnName, String newColumnName) { - Table oldTable = getTableOrElseThrow(databaseName, tableName); + Table oldTable = getTableOrElseThrow(identity, databaseName, tableName); if (oldTable.getPartitionColumns().stream().anyMatch(c -> c.getName().equals(oldColumnName))) { throw new PrestoException(NOT_SUPPORTED, "Renaming partition columns is not supported"); } @@ -594,14 +595,14 @@ public void renameColumn(String databaseName, String tableName, String oldColumn Table newTable = Table.builder(oldTable) .setDataColumns(newDataColumns.build()) .build(); - replaceTable(databaseName, tableName, newTable, null); + replaceTable(identity, databaseName, tableName, newTable, null); } @Override - public void dropColumn(String databaseName, String tableName, String columnName) + public void dropColumn(HiveIdentity identity, String databaseName, String tableName, String columnName) { - verifyCanDropColumn(this, databaseName, tableName, columnName); - Table oldTable = getTableOrElseThrow(databaseName, tableName); + verifyCanDropColumn(this, identity, databaseName, tableName, columnName); + Table oldTable = getTableOrElseThrow(identity, databaseName, tableName); if (!oldTable.getColumn(columnName).isPresent()) { SchemaTableName name = new SchemaTableName(databaseName, tableName); @@ -616,11 +617,11 @@ public void dropColumn(String databaseName, String tableName, String columnName) Table newTable = Table.builder(oldTable) .setDataColumns(newDataColumns.build()) .build(); - replaceTable(databaseName, tableName, newTable, null); + replaceTable(identity, databaseName, tableName, newTable, null); } @Override - public Optional getPartition(String databaseName, String tableName, List partitionValues) + public Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues) { try { GetPartitionResult result = glueClient.getPartition(new GetPartitionRequest() @@ -639,9 +640,9 @@ public Optional getPartition(String databaseName, String tableName, L } @Override - public Optional> getPartitionNames(String databaseName, String tableName) + public Optional> getPartitionNames(HiveIdentity identity, String databaseName, String tableName) { - Table table = getTableOrElseThrow(databaseName, tableName); + Table table = getTableOrElseThrow(identity, databaseName, tableName); List partitions = getPartitions(databaseName, tableName, WILDCARD_EXPRESSION); return Optional.of(buildPartitionNames(table.getPartitionColumns(), partitions)); } @@ -658,9 +659,9 @@ public Optional> getPartitionNames(String databaseName, String tabl * @return a list of partition names. */ @Override - public Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) + public Optional> getPartitionNamesByParts(HiveIdentity identity, String databaseName, String tableName, List parts) { - Table table = getTableOrElseThrow(databaseName, tableName); + Table table = getTableOrElseThrow(identity, databaseName, tableName); String expression = buildGlueExpression(table.getPartitionColumns(), parts); List partitions = getPartitions(databaseName, tableName, expression); return Optional.of(buildPartitionNames(table.getPartitionColumns(), partitions)); @@ -709,7 +710,7 @@ private static List buildPartitionNames(List partitionColumns, L * @return Mapping of partition name to partition object */ @Override - public Map> getPartitionsByNames(String databaseName, String tableName, List partitionNames) + public Map> getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames) { requireNonNull(partitionNames, "partitionNames is null"); if (partitionNames.isEmpty()) { @@ -765,7 +766,7 @@ private List batchGetPartition(String databaseName, String tableName, } @Override - public void addPartitions(String databaseName, String tableName, List partitions) + public void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitions) { try { List> batchedPartitions = Lists.partition(partitions, BATCH_CREATE_PARTITION_MAX_PAGE_SIZE); @@ -811,10 +812,10 @@ private static void propagatePartitionErrorToPrestoException(String databaseName } @Override - public void dropPartition(String databaseName, String tableName, List parts, boolean deleteData) + public void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData) { - Table table = getTableOrElseThrow(databaseName, tableName); - Partition partition = getPartition(databaseName, tableName, parts) + Table table = getTableOrElseThrow(identity, databaseName, tableName); + Partition partition = getPartition(identity, databaseName, tableName, parts) .orElseThrow(() -> new PartitionNotFoundException(new SchemaTableName(databaseName, tableName), parts)); try { @@ -835,7 +836,7 @@ public void dropPartition(String databaseName, String tableName, List pa } @Override - public void alterPartition(String databaseName, String tableName, PartitionWithStatistics partition) + public void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partition) { try { PartitionInput newPartition = GlueInputConverter.convertPartition(partition); @@ -910,4 +911,10 @@ public Set listTablePrivileges(String databaseName, String ta { throw new PrestoException(NOT_SUPPORTED, "listTablePrivileges is not supported by Glue"); } + + @Override + public boolean isImpersonationEnabled() + { + throw new PrestoException(NOT_SUPPORTED, "isImpersonationEnabled 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 ada7a46494320..a0d599c1e4f3e 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 @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableMap; import io.prestosql.plugin.hive.HiveType; import io.prestosql.plugin.hive.PartitionStatistics; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Database; import io.prestosql.plugin.hive.metastore.HiveMetastore; import io.prestosql.plugin.hive.metastore.HivePrincipal; @@ -79,11 +80,11 @@ public List getAllDatabases() } @Override - public Optional
getTable(String databaseName, String tableName) + public Optional
getTable(HiveIdentity identity, String databaseName, String tableName) { - return delegate.getTable(databaseName, tableName).map(table -> { + return delegate.getTable(identity, databaseName, tableName).map(table -> { if (isAvroTableWithSchemaSet(table) || isCsvTable(table)) { - return fromMetastoreApiTable(table, delegate.getFields(databaseName, tableName).get()); + return fromMetastoreApiTable(table, delegate.getFields(identity, databaseName, tableName).get()); } return fromMetastoreApiTable(table); }); @@ -96,27 +97,27 @@ public Set getSupportedColumnStatistics(Type type) } @Override - public PartitionStatistics getTableStatistics(String databaseName, String tableName) + public PartitionStatistics getTableStatistics(HiveIdentity identity, String databaseName, String tableName) { - return delegate.getTableStatistics(databaseName, tableName); + return delegate.getTableStatistics(identity, databaseName, tableName); } @Override - public Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames) + public Map getPartitionStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames) { - return delegate.getPartitionStatistics(databaseName, tableName, partitionNames); + return delegate.getPartitionStatistics(identity, databaseName, tableName, partitionNames); } @Override - public void updateTableStatistics(String databaseName, String tableName, Function update) + public void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, Function update) { - delegate.updateTableStatistics(databaseName, tableName, update); + delegate.updateTableStatistics(identity, databaseName, tableName, update); } @Override - public void updatePartitionStatistics(String databaseName, String tableName, String partitionName, Function update) + public void updatePartitionStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, Function update) { - delegate.updatePartitionStatistics(databaseName, tableName, partitionName, update); + delegate.updatePartitionStatistics(identity, databaseName, tableName, partitionName, update); } @Override @@ -138,24 +139,24 @@ public List getAllViews(String databaseName) } @Override - public void createDatabase(Database database) + public void createDatabase(HiveIdentity identity, Database database) { - delegate.createDatabase(toMetastoreApiDatabase(database)); + delegate.createDatabase(identity, toMetastoreApiDatabase(database)); } @Override - public void dropDatabase(String databaseName) + public void dropDatabase(HiveIdentity identity, String databaseName) { - delegate.dropDatabase(databaseName); + delegate.dropDatabase(identity, databaseName); } @Override - public void renameDatabase(String databaseName, String newDatabaseName) + public void renameDatabase(HiveIdentity identity, String databaseName, String newDatabaseName) { org.apache.hadoop.hive.metastore.api.Database database = delegate.getDatabase(databaseName) .orElseThrow(() -> new SchemaNotFoundException(databaseName)); database.setName(newDatabaseName); - delegate.alterDatabase(databaseName, database); + delegate.alterDatabase(identity, databaseName, database); delegate.getDatabase(databaseName).ifPresent(newDatabase -> { if (newDatabase.getName().equals(databaseName)) { @@ -165,40 +166,40 @@ public void renameDatabase(String databaseName, String newDatabaseName) } @Override - public void createTable(Table table, PrincipalPrivileges principalPrivileges) + public void createTable(HiveIdentity identity, Table table, PrincipalPrivileges principalPrivileges) { - delegate.createTable(toMetastoreApiTable(table, principalPrivileges)); + delegate.createTable(identity, toMetastoreApiTable(table, principalPrivileges)); } @Override - public void dropTable(String databaseName, String tableName, boolean deleteData) + public void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) { - delegate.dropTable(databaseName, tableName, deleteData); + delegate.dropTable(identity, databaseName, tableName, deleteData); } @Override - public void replaceTable(String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) + public void replaceTable(HiveIdentity identity, String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) { - alterTable(databaseName, tableName, toMetastoreApiTable(newTable, principalPrivileges)); + alterTable(identity, databaseName, tableName, toMetastoreApiTable(newTable, principalPrivileges)); } @Override - public void renameTable(String databaseName, String tableName, String newDatabaseName, String newTableName) + public void renameTable(HiveIdentity identity, String databaseName, String tableName, String newDatabaseName, String newTableName) { - Optional source = delegate.getTable(databaseName, tableName); + Optional source = delegate.getTable(identity, databaseName, tableName); if (!source.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } org.apache.hadoop.hive.metastore.api.Table table = source.get(); table.setDbName(newDatabaseName); table.setTableName(newTableName); - alterTable(databaseName, tableName, table); + alterTable(identity, databaseName, tableName, table); } @Override - public void commentTable(String databaseName, String tableName, Optional comment) + public void commentTable(HiveIdentity identity, String databaseName, String tableName, Optional comment) { - Optional source = delegate.getTable(databaseName, tableName); + Optional source = delegate.getTable(identity, databaseName, tableName); if (!source.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -210,26 +211,26 @@ public void commentTable(String databaseName, String tableName, Optional comment.ifPresent(value -> parameters.put(TABLE_COMMENT, comment.get())); table.setParameters(parameters); - alterTable(databaseName, tableName, table); + alterTable(identity, databaseName, tableName, table); } @Override - public void addColumn(String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) + public void addColumn(HiveIdentity identity, String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) { - Optional source = delegate.getTable(databaseName, tableName); + Optional source = delegate.getTable(identity, databaseName, tableName); if (!source.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } org.apache.hadoop.hive.metastore.api.Table table = source.get(); table.getSd().getCols().add( new FieldSchema(columnName, columnType.getHiveTypeName().toString(), columnComment)); - alterTable(databaseName, tableName, table); + alterTable(identity, databaseName, tableName, table); } @Override - public void renameColumn(String databaseName, String tableName, String oldColumnName, String newColumnName) + public void renameColumn(HiveIdentity identity, String databaseName, String tableName, String oldColumnName, String newColumnName) { - Optional source = delegate.getTable(databaseName, tableName); + Optional source = delegate.getTable(identity, databaseName, tableName); if (!source.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -244,44 +245,44 @@ public void renameColumn(String databaseName, String tableName, String oldColumn fieldSchema.setName(newColumnName); } } - alterTable(databaseName, tableName, table); + alterTable(identity, databaseName, tableName, table); } @Override - public void dropColumn(String databaseName, String tableName, String columnName) + public void dropColumn(HiveIdentity identity, String databaseName, String tableName, String columnName) { - verifyCanDropColumn(this, databaseName, tableName, columnName); - org.apache.hadoop.hive.metastore.api.Table table = delegate.getTable(databaseName, tableName) + verifyCanDropColumn(this, identity, databaseName, tableName, columnName); + org.apache.hadoop.hive.metastore.api.Table table = delegate.getTable(identity, databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); table.getSd().getCols().removeIf(fieldSchema -> fieldSchema.getName().equals(columnName)); - alterTable(databaseName, tableName, table); + alterTable(identity, databaseName, tableName, table); } - private void alterTable(String databaseName, String tableName, org.apache.hadoop.hive.metastore.api.Table table) + private void alterTable(HiveIdentity identity, String databaseName, String tableName, org.apache.hadoop.hive.metastore.api.Table table) { - delegate.alterTable(databaseName, tableName, table); + delegate.alterTable(identity, databaseName, tableName, table); } @Override - public Optional getPartition(String databaseName, String tableName, List partitionValues) + public Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues) { - return delegate.getPartition(databaseName, tableName, partitionValues).map(ThriftMetastoreUtil::fromMetastoreApiPartition); + return delegate.getPartition(identity, databaseName, tableName, partitionValues).map(ThriftMetastoreUtil::fromMetastoreApiPartition); } @Override - public Optional> getPartitionNames(String databaseName, String tableName) + public Optional> getPartitionNames(HiveIdentity identity, String databaseName, String tableName) { - return delegate.getPartitionNames(databaseName, tableName); + return delegate.getPartitionNames(identity, databaseName, tableName); } @Override - public Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) + public Optional> getPartitionNamesByParts(HiveIdentity identity, String databaseName, String tableName, List parts) { - return delegate.getPartitionNamesByParts(databaseName, tableName, parts); + return delegate.getPartitionNamesByParts(identity, databaseName, tableName, parts); } @Override - public Map> getPartitionsByNames(String databaseName, String tableName, List partitionNames) + public Map> getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames) { requireNonNull(partitionNames, "partitionNames is null"); if (partitionNames.isEmpty()) { @@ -289,17 +290,17 @@ public Map> getPartitionsByNames(String databaseName } Function fromMetastoreApiPartition = ThriftMetastoreUtil::fromMetastoreApiPartition; - boolean isAvroTableWithSchemaSet = delegate.getTable(databaseName, tableName) + boolean isAvroTableWithSchemaSet = delegate.getTable(identity, databaseName, tableName) .map(ThriftMetastoreUtil::isAvroTableWithSchemaSet) .orElse(false); if (isAvroTableWithSchemaSet) { - List schema = delegate.getFields(databaseName, tableName).get(); + List schema = delegate.getFields(identity, databaseName, tableName).get(); fromMetastoreApiPartition = partition -> fromMetastoreApiPartition(partition, schema); } Map> partitionNameToPartitionValuesMap = partitionNames.stream() .collect(Collectors.toMap(identity(), HiveUtil::toPartitionValues)); - Map, Partition> partitionValuesToPartitionMap = delegate.getPartitionsByNames(databaseName, tableName, partitionNames).stream() + Map, Partition> partitionValuesToPartitionMap = delegate.getPartitionsByNames(identity, databaseName, tableName, partitionNames).stream() .map(fromMetastoreApiPartition) .collect(Collectors.toMap(Partition::getValues, identity())); ImmutableMap.Builder> resultBuilder = ImmutableMap.builder(); @@ -311,21 +312,21 @@ public Map> getPartitionsByNames(String databaseName } @Override - public void addPartitions(String databaseName, String tableName, List partitions) + public void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitions) { - delegate.addPartitions(databaseName, tableName, partitions); + delegate.addPartitions(identity, databaseName, tableName, partitions); } @Override - public void dropPartition(String databaseName, String tableName, List parts, boolean deleteData) + public void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData) { - delegate.dropPartition(databaseName, tableName, parts, deleteData); + delegate.dropPartition(identity, databaseName, tableName, parts, deleteData); } @Override - public void alterPartition(String databaseName, String tableName, PartitionWithStatistics partition) + public void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partition) { - delegate.alterPartition(databaseName, tableName, partition); + delegate.alterPartition(identity, databaseName, tableName, partition); } @Override @@ -381,4 +382,10 @@ public Set listTablePrivileges(String databaseName, String ta { return delegate.listTablePrivileges(databaseName, tableName, tableOwner, principal); } + + @Override + public boolean isImpersonationEnabled() + { + return delegate.isImpersonationEnabled(); + } } 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 4fac45b20c7da..60855d462549b 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 @@ -25,6 +25,7 @@ import io.prestosql.plugin.hive.PartitionStatistics; import io.prestosql.plugin.hive.SchemaAlreadyExistsException; import io.prestosql.plugin.hive.TableAlreadyExistsException; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Column; import io.prestosql.plugin.hive.metastore.HiveColumnStatistics; import io.prestosql.plugin.hive.metastore.HivePrincipal; @@ -64,6 +65,8 @@ import javax.annotation.concurrent.ThreadSafe; import javax.inject.Inject; +import java.io.Closeable; +import java.io.IOException; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -119,6 +122,7 @@ public class ThriftHiveMetastore private final Duration maxBackoffDelay; private final Duration maxRetryTime; private final int maxRetries; + private final boolean impersonationEnabled; private final AtomicInteger chosenGetTableAlternative = new AtomicInteger(Integer.MAX_VALUE); private final AtomicInteger chosenTableParamAlternative = new AtomicInteger(Integer.MAX_VALUE); @@ -135,6 +139,7 @@ public ThriftHiveMetastore(MetastoreLocator metastoreLocator, ThriftHiveMetastor this.maxBackoffDelay = thriftConfig.getMaxBackoffDelay(); this.maxRetryTime = thriftConfig.getMaxRetryTime(); this.maxRetries = thriftConfig.getMaxRetries(); + this.impersonationEnabled = thriftConfig.isImpersonationEnabled(); } @Managed @@ -234,14 +239,14 @@ public List getTablesWithParameter(String databaseName, String parameter } @Override - public Optional
getTable(String databaseName, String tableName) + public Optional
getTable(HiveIdentity identity, String databaseName, String tableName) { try { return retry() .stopOn(NoSuchObjectException.class, HiveViewNotSupportedException.class) .stopOnIllegalExceptions() .run("getTable", stats.getGetTable().wrap(() -> { - Table table = getTableFromMetastore(databaseName, tableName); + Table table = getTableFromMetastore(identity, databaseName, tableName); if (table.getTableType().equals(TableType.VIRTUAL_VIEW.name()) && !isPrestoView(table)) { throw new HiveViewNotSupportedException(new SchemaTableName(databaseName, tableName)); } @@ -259,10 +264,11 @@ public Optional
getTable(String databaseName, String tableName) } } - private Table getTableFromMetastore(String databaseName, String tableName) + private Table getTableFromMetastore(HiveIdentity identity, String databaseName, String tableName) throws TException { return alternativeCall( + () -> createMetastoreClient(identity), chosenGetTableAlternative, client -> client.getTable(databaseName, tableName), client -> client.getTableWithCapabilities(databaseName, tableName)); @@ -280,26 +286,26 @@ private static boolean isPrestoView(Table table) } @Override - public PartitionStatistics getTableStatistics(String databaseName, String tableName) + public PartitionStatistics getTableStatistics(HiveIdentity identity, String databaseName, String tableName) { - Table table = getTable(databaseName, tableName) + Table table = getTable(identity, databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); List dataColumns = table.getSd().getCols().stream() .map(FieldSchema::getName) .collect(toImmutableList()); HiveBasicStatistics basicStatistics = getHiveBasicStatistics(table.getParameters()); - Map columnStatistics = getTableColumnStatistics(databaseName, tableName, dataColumns, basicStatistics.getRowCount()); + Map columnStatistics = getTableColumnStatistics(identity, databaseName, tableName, dataColumns, basicStatistics.getRowCount()); return new PartitionStatistics(basicStatistics, columnStatistics); } - private Map getTableColumnStatistics(String databaseName, String tableName, List columns, OptionalLong rowCount) + private Map getTableColumnStatistics(HiveIdentity identity, String databaseName, String tableName, List columns, OptionalLong rowCount) { try { return retry() .stopOn(NoSuchObjectException.class, HiveViewNotSupportedException.class) .stopOnIllegalExceptions() .run("getTableColumnStatistics", stats.getGetTableColumnStatistics().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { return groupStatisticsByColumn(client.getTableColumnStatistics(databaseName, tableName, columns), rowCount); } })); @@ -316,9 +322,9 @@ private Map getTableColumnStatistics(String databa } @Override - public Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames) + public Map getPartitionStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames) { - Table table = getTable(databaseName, tableName) + Table table = getTable(identity, databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); List dataColumns = table.getSd().getCols().stream() .map(FieldSchema::getName) @@ -327,13 +333,14 @@ public Map getPartitionStatistics(String databaseNa .map(FieldSchema::getName) .collect(toImmutableList()); - Map partitionBasicStatistics = getPartitionsByNames(databaseName, tableName, ImmutableList.copyOf(partitionNames)).stream() + Map partitionBasicStatistics = getPartitionsByNames(identity, databaseName, tableName, ImmutableList.copyOf(partitionNames)).stream() .collect(toImmutableMap( partition -> makePartName(partitionColumns, partition.getValues()), partition -> getHiveBasicStatistics(partition.getParameters()))); Map partitionRowCounts = partitionBasicStatistics.entrySet().stream() .collect(toImmutableMap(Map.Entry::getKey, entry -> entry.getValue().getRowCount())); Map> partitionColumnStatistics = getPartitionColumnStatistics( + identity, databaseName, tableName, partitionNames, @@ -350,7 +357,7 @@ public Map getPartitionStatistics(String databaseNa } @Override - public Optional> getFields(String databaseName, String tableName) + public Optional> getFields(HiveIdentity identity, String databaseName, String tableName) { try { return retry() @@ -374,27 +381,28 @@ public Optional> getFields(String databaseName, String tableNa } private Map> getPartitionColumnStatistics( + HiveIdentity identity, String databaseName, String tableName, Set partitionNames, List columnNames, Map partitionRowCounts) { - return getMetastorePartitionColumnStatistics(databaseName, tableName, partitionNames, columnNames).entrySet().stream() + return getMetastorePartitionColumnStatistics(identity, databaseName, tableName, partitionNames, columnNames).entrySet().stream() .filter(entry -> !entry.getValue().isEmpty()) .collect(toImmutableMap( Map.Entry::getKey, entry -> groupStatisticsByColumn(entry.getValue(), partitionRowCounts.getOrDefault(entry.getKey(), OptionalLong.empty())))); } - private Map> getMetastorePartitionColumnStatistics(String databaseName, String tableName, Set partitionNames, List columnNames) + private Map> getMetastorePartitionColumnStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames, List columnNames) { try { return retry() .stopOn(NoSuchObjectException.class, HiveViewNotSupportedException.class) .stopOnIllegalExceptions() .run("getPartitionColumnStatistics", stats.getGetPartitionColumnStatistics().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { return client.getPartitionColumnStatistics(databaseName, tableName, ImmutableList.copyOf(partitionNames), columnNames); } })); @@ -417,17 +425,17 @@ private Map groupStatisticsByColumn(List update) + public synchronized void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, Function update) { - PartitionStatistics currentStatistics = getTableStatistics(databaseName, tableName); + PartitionStatistics currentStatistics = getTableStatistics(identity, databaseName, tableName); PartitionStatistics updatedStatistics = update.apply(currentStatistics); - Table originalTable = getTable(databaseName, tableName) + Table originalTable = getTable(identity, databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); Table modifiedTable = originalTable.deepCopy(); HiveBasicStatistics basicStatistics = updatedStatistics.getBasicStatistics(); modifiedTable.setParameters(updateStatisticsParameters(modifiedTable.getParameters(), basicStatistics)); - alterTable(databaseName, tableName, modifiedTable); + alterTable(identity, databaseName, tableName, modifiedTable); io.prestosql.plugin.hive.metastore.Table table = fromMetastoreApiTable(modifiedTable); OptionalLong rowCount = basicStatistics.getRowCount(); @@ -435,20 +443,20 @@ public synchronized void updateTableStatistics(String databaseName, String table .map(entry -> createMetastoreColumnStatistics(entry.getKey(), table.getColumn(entry.getKey()).get().getType(), entry.getValue(), rowCount)) .collect(toImmutableList()); if (!metastoreColumnStatistics.isEmpty()) { - setTableColumnStatistics(databaseName, tableName, metastoreColumnStatistics); + setTableColumnStatistics(identity, databaseName, tableName, metastoreColumnStatistics); } Set removedColumnStatistics = difference(currentStatistics.getColumnStatistics().keySet(), updatedStatistics.getColumnStatistics().keySet()); - removedColumnStatistics.forEach(column -> deleteTableColumnStatistics(databaseName, tableName, column)); + removedColumnStatistics.forEach(column -> deleteTableColumnStatistics(identity, databaseName, tableName, column)); } - private void setTableColumnStatistics(String databaseName, String tableName, List statistics) + private void setTableColumnStatistics(HiveIdentity identity, String databaseName, String tableName, List statistics) { try { retry() .stopOn(NoSuchObjectException.class, InvalidObjectException.class, MetaException.class, InvalidInputException.class) .stopOnIllegalExceptions() .run("setTableColumnStatistics", stats.getCreateDatabase().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { client.setTableColumnStatistics(databaseName, tableName, statistics); } return null; @@ -465,14 +473,14 @@ private void setTableColumnStatistics(String databaseName, String tableName, Lis } } - private void deleteTableColumnStatistics(String databaseName, String tableName, String columnName) + private void deleteTableColumnStatistics(HiveIdentity identity, String databaseName, String tableName, String columnName) { try { retry() .stopOn(NoSuchObjectException.class, InvalidObjectException.class, MetaException.class, InvalidInputException.class) .stopOnIllegalExceptions() .run("deleteTableColumnStatistics", stats.getCreateDatabase().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { client.deleteTableColumnStatistics(databaseName, tableName, columnName); } return null; @@ -490,13 +498,13 @@ private void deleteTableColumnStatistics(String databaseName, String tableName, } @Override - public synchronized void updatePartitionStatistics(String databaseName, String tableName, String partitionName, Function update) + public synchronized void updatePartitionStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, Function update) { PartitionStatistics currentStatistics = requireNonNull( - getPartitionStatistics(databaseName, tableName, ImmutableSet.of(partitionName)).get(partitionName), "getPartitionStatistics() returned null"); + getPartitionStatistics(identity, databaseName, tableName, ImmutableSet.of(partitionName)).get(partitionName), "getPartitionStatistics() returned null"); PartitionStatistics updatedStatistics = update.apply(currentStatistics); - List partitions = getPartitionsByNames(databaseName, tableName, ImmutableList.of(partitionName)); + List partitions = getPartitionsByNames(identity, databaseName, tableName, ImmutableList.of(partitionName)); if (partitions.size() != 1) { throw new PrestoException(HIVE_METASTORE_ERROR, "Metastore returned multiple partitions for name: " + partitionName); } @@ -505,17 +513,18 @@ public synchronized void updatePartitionStatistics(String databaseName, String t Partition modifiedPartition = originalPartition.deepCopy(); HiveBasicStatistics basicStatistics = updatedStatistics.getBasicStatistics(); modifiedPartition.setParameters(updateStatisticsParameters(modifiedPartition.getParameters(), basicStatistics)); - alterPartitionWithoutStatistics(databaseName, tableName, modifiedPartition); + alterPartitionWithoutStatistics(identity, databaseName, tableName, modifiedPartition); Map columns = modifiedPartition.getSd().getCols().stream() .collect(toImmutableMap(FieldSchema::getName, schema -> HiveType.valueOf(schema.getType()))); - setPartitionColumnStatistics(databaseName, tableName, partitionName, columns, updatedStatistics.getColumnStatistics(), basicStatistics.getRowCount()); + setPartitionColumnStatistics(identity, databaseName, tableName, partitionName, columns, updatedStatistics.getColumnStatistics(), basicStatistics.getRowCount()); Set removedStatistics = difference(currentStatistics.getColumnStatistics().keySet(), updatedStatistics.getColumnStatistics().keySet()); - removedStatistics.forEach(column -> deletePartitionColumnStatistics(databaseName, tableName, partitionName, column)); + removedStatistics.forEach(column -> deletePartitionColumnStatistics(identity, databaseName, tableName, partitionName, column)); } private void setPartitionColumnStatistics( + HiveIdentity identity, String databaseName, String tableName, String partitionName, @@ -528,18 +537,18 @@ private void setPartitionColumnStatistics( .map(entry -> createMetastoreColumnStatistics(entry.getKey(), columns.get(entry.getKey()), entry.getValue(), rowCount)) .collect(toImmutableList()); if (!metastoreColumnStatistics.isEmpty()) { - setPartitionColumnStatistics(databaseName, tableName, partitionName, metastoreColumnStatistics); + setPartitionColumnStatistics(identity, databaseName, tableName, partitionName, metastoreColumnStatistics); } } - private void setPartitionColumnStatistics(String databaseName, String tableName, String partitionName, List statistics) + private void setPartitionColumnStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, List statistics) { try { retry() .stopOn(NoSuchObjectException.class, InvalidObjectException.class, MetaException.class, InvalidInputException.class) .stopOnIllegalExceptions() .run("setPartitionColumnStatistics", stats.getCreateDatabase().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { client.setPartitionColumnStatistics(databaseName, tableName, partitionName, statistics); } return null; @@ -556,14 +565,14 @@ private void setPartitionColumnStatistics(String databaseName, String tableName, } } - private void deletePartitionColumnStatistics(String databaseName, String tableName, String partitionName, String columnName) + private void deletePartitionColumnStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, String columnName) { try { retry() .stopOn(NoSuchObjectException.class, InvalidObjectException.class, MetaException.class, InvalidInputException.class) .stopOnIllegalExceptions() .run("deletePartitionColumnStatistics", stats.getCreateDatabase().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { client.deletePartitionColumnStatistics(databaseName, tableName, partitionName, columnName); } return null; @@ -777,20 +786,21 @@ private List doGetTablesWithParameter(String databaseName, String parame String filterWithLike = HIVE_FILTER_FIELD_PARAMS + parameterKey + " LIKE \"" + parameterValue + "\""; return alternativeCall( + this::createMetastoreClient, chosenTableParamAlternative, client -> client.getTableNamesByFilter(databaseName, filterWithEquals), client -> client.getTableNamesByFilter(databaseName, filterWithLike)); } @Override - public void createDatabase(Database database) + public void createDatabase(HiveIdentity identity, Database database) { try { retry() .stopOn(AlreadyExistsException.class, InvalidObjectException.class, MetaException.class) .stopOnIllegalExceptions() .run("createDatabase", stats.getCreateDatabase().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { client.createDatabase(database); } return null; @@ -808,14 +818,14 @@ public void createDatabase(Database database) } @Override - public void dropDatabase(String databaseName) + public void dropDatabase(HiveIdentity identity, String databaseName) { try { retry() .stopOn(NoSuchObjectException.class, InvalidOperationException.class) .stopOnIllegalExceptions() .run("dropDatabase", stats.getDropDatabase().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { client.dropDatabase(databaseName, false, false); } return null; @@ -833,14 +843,14 @@ public void dropDatabase(String databaseName) } @Override - public void alterDatabase(String databaseName, Database database) + public void alterDatabase(HiveIdentity identity, String databaseName, Database database) { try { retry() .stopOn(NoSuchObjectException.class, MetaException.class) .stopOnIllegalExceptions() .run("alterDatabase", stats.getAlterDatabase().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { client.alterDatabase(databaseName, database); } return null; @@ -858,14 +868,14 @@ public void alterDatabase(String databaseName, Database database) } @Override - public void createTable(Table table) + public void createTable(HiveIdentity identity, Table table) { try { retry() .stopOn(AlreadyExistsException.class, InvalidObjectException.class, MetaException.class, NoSuchObjectException.class) .stopOnIllegalExceptions() .run("createTable", stats.getCreateTable().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { client.createTable(table); } return null; @@ -886,14 +896,14 @@ public void createTable(Table table) } @Override - public void dropTable(String databaseName, String tableName, boolean deleteData) + public void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) { try { retry() .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("dropTable", stats.getDropTable().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { client.dropTable(databaseName, tableName, deleteData); } return null; @@ -911,15 +921,15 @@ public void dropTable(String databaseName, String tableName, boolean deleteData) } @Override - public void alterTable(String databaseName, String tableName, Table table) + public void alterTable(HiveIdentity identity, String databaseName, String tableName, Table table) { try { retry() .stopOn(InvalidOperationException.class, MetaException.class) .stopOnIllegalExceptions() .run("alterTable", stats.getAlterTable().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { - Optional
source = getTable(databaseName, tableName); + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + Optional
source = getTable(identity, databaseName, tableName); if (!source.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -940,14 +950,14 @@ public void alterTable(String databaseName, String tableName, Table table) } @Override - public Optional> getPartitionNames(String databaseName, String tableName) + public Optional> getPartitionNames(HiveIdentity identity, String databaseName, String tableName) { try { return retry() .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getPartitionNames", stats.getGetPartitionNames().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { return Optional.of(client.getPartitionNames(databaseName, tableName)); } })); @@ -964,14 +974,14 @@ public Optional> getPartitionNames(String databaseName, String tabl } @Override - public Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) + public Optional> getPartitionNamesByParts(HiveIdentity identity, String databaseName, String tableName, List parts) { try { return retry() .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getPartitionNamesByParts", stats.getGetPartitionNamesPs().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { return Optional.of(client.getPartitionNamesFiltered(databaseName, tableName, parts)); } })); @@ -988,18 +998,18 @@ public Optional> getPartitionNamesByParts(String databaseName, Stri } @Override - public void addPartitions(String databaseName, String tableName, List partitionsWithStatistics) + public void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitionsWithStatistics) { List partitions = partitionsWithStatistics.stream() .map(ThriftMetastoreUtil::toMetastoreApiPartition) .collect(toImmutableList()); - addPartitionsWithoutStatistics(databaseName, tableName, partitions); + addPartitionsWithoutStatistics(identity, databaseName, tableName, partitions); for (PartitionWithStatistics partitionWithStatistics : partitionsWithStatistics) { - storePartitionColumnStatistics(databaseName, tableName, partitionWithStatistics.getPartitionName(), partitionWithStatistics); + storePartitionColumnStatistics(identity, databaseName, tableName, partitionWithStatistics.getPartitionName(), partitionWithStatistics); } } - private void addPartitionsWithoutStatistics(String databaseName, String tableName, List partitions) + private void addPartitionsWithoutStatistics(HiveIdentity identity, String databaseName, String tableName, List partitions) { if (partitions.isEmpty()) { return; @@ -1009,7 +1019,7 @@ private void addPartitionsWithoutStatistics(String databaseName, String tableNam .stopOn(AlreadyExistsException.class, InvalidObjectException.class, MetaException.class, NoSuchObjectException.class, PrestoException.class) .stopOnIllegalExceptions() .run("addPartitions", stats.getAddPartitions().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { int partitionsAdded = client.addPartitions(partitions); if (partitionsAdded != partitions.size()) { throw new PrestoException(HIVE_METASTORE_ERROR, @@ -1034,14 +1044,14 @@ private void addPartitionsWithoutStatistics(String databaseName, String tableNam } @Override - public void dropPartition(String databaseName, String tableName, List parts, boolean deleteData) + public void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData) { try { retry() .stopOn(NoSuchObjectException.class, MetaException.class) .stopOnIllegalExceptions() .run("dropPartition", stats.getDropPartition().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { client.dropPartition(databaseName, tableName, parts, deleteData); } return null; @@ -1059,21 +1069,21 @@ public void dropPartition(String databaseName, String tableName, List pa } @Override - public void alterPartition(String databaseName, String tableName, PartitionWithStatistics partitionWithStatistics) + public void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partitionWithStatistics) { - alterPartitionWithoutStatistics(databaseName, tableName, toMetastoreApiPartition(partitionWithStatistics)); - storePartitionColumnStatistics(databaseName, tableName, partitionWithStatistics.getPartitionName(), partitionWithStatistics); - dropExtraColumnStatisticsAfterAlterPartition(databaseName, tableName, partitionWithStatistics); + alterPartitionWithoutStatistics(identity, databaseName, tableName, toMetastoreApiPartition(partitionWithStatistics)); + storePartitionColumnStatistics(identity, databaseName, tableName, partitionWithStatistics.getPartitionName(), partitionWithStatistics); + dropExtraColumnStatisticsAfterAlterPartition(identity, databaseName, tableName, partitionWithStatistics); } - private void alterPartitionWithoutStatistics(String databaseName, String tableName, Partition partition) + private void alterPartitionWithoutStatistics(HiveIdentity identity, String databaseName, String tableName, Partition partition) { try { retry() .stopOn(NoSuchObjectException.class, MetaException.class) .stopOnIllegalExceptions() .run("alterPartition", stats.getAlterPartition().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { client.alterPartition(databaseName, tableName, partition); } return null; @@ -1090,7 +1100,7 @@ private void alterPartitionWithoutStatistics(String databaseName, String tableNa } } - private void storePartitionColumnStatistics(String databaseName, String tableName, String partitionName, PartitionWithStatistics partitionWithStatistics) + private void storePartitionColumnStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, PartitionWithStatistics partitionWithStatistics) { PartitionStatistics statistics = partitionWithStatistics.getStatistics(); Map columnStatistics = statistics.getColumnStatistics(); @@ -1099,7 +1109,7 @@ private void storePartitionColumnStatistics(String databaseName, String tableNam } Map columnTypes = partitionWithStatistics.getPartition().getColumns().stream() .collect(toImmutableMap(Column::getName, Column::getType)); - setPartitionColumnStatistics(databaseName, tableName, partitionName, columnTypes, columnStatistics, statistics.getBasicStatistics().getRowCount()); + setPartitionColumnStatistics(identity, databaseName, tableName, partitionName, columnTypes, columnStatistics, statistics.getBasicStatistics().getRowCount()); } /* @@ -1111,6 +1121,7 @@ private void storePartitionColumnStatistics(String databaseName, String tableNam * if is needed to explicitly remove the statistics from the metastore for that columns. */ private void dropExtraColumnStatisticsAfterAlterPartition( + HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partitionWithStatistics) @@ -1132,6 +1143,7 @@ private void dropExtraColumnStatisticsAfterAlterPartition( // when trying to remove any missing statistics the metastore throws NoSuchObjectException String partitionName = partitionWithStatistics.getPartitionName(); List statisticsToBeRemoved = getMetastorePartitionColumnStatistics( + identity, databaseName, tableName, ImmutableSet.of(partitionName), @@ -1139,12 +1151,12 @@ private void dropExtraColumnStatisticsAfterAlterPartition( .getOrDefault(partitionName, ImmutableList.of()); for (ColumnStatisticsObj statistics : statisticsToBeRemoved) { - deletePartitionColumnStatistics(databaseName, tableName, partitionName, statistics.getColName()); + deletePartitionColumnStatistics(identity, databaseName, tableName, partitionName, statistics.getColName()); } } @Override - public Optional getPartition(String databaseName, String tableName, List partitionValues) + public Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues) { requireNonNull(partitionValues, "partitionValues is null"); try { @@ -1152,7 +1164,7 @@ public Optional getPartition(String databaseName, String tableName, L .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getPartition", stats.getGetPartition().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { return Optional.of(client.getPartition(databaseName, tableName, partitionValues)); } })); @@ -1169,7 +1181,7 @@ public Optional getPartition(String databaseName, String tableName, L } @Override - public List getPartitionsByNames(String databaseName, String tableName, List partitionNames) + public List getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames) { requireNonNull(partitionNames, "partitionNames is null"); checkArgument(!Iterables.isEmpty(partitionNames), "partitionNames is empty"); @@ -1179,7 +1191,7 @@ public List getPartitionsByNames(String databaseName, String tableNam .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getPartitionsByNames", stats.getGetPartitionsByNames().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = createMetastoreClient(identity)) { return client.getPartitionsByNames(databaseName, tableName, partitionNames); } })); @@ -1292,7 +1304,6 @@ public Set listTablePrivileges(String databaseName, String ta return retry() .stopOnIllegalExceptions() .run("listTablePrivileges", stats.getListTablePrivileges().wrap(() -> { - Table table = getTableFromMetastore(databaseName, tableName); try (ThriftMetastoreClient client = createMetastoreClient()) { ImmutableSet.Builder privileges = ImmutableSet.builder(); List hiveObjectPrivilegeList; @@ -1328,6 +1339,12 @@ public Set listTablePrivileges(String databaseName, String ta } } + @Override + public boolean isImpersonationEnabled() + { + return impersonationEnabled; + } + private PrivilegeBag buildPrivilegeBag( String databaseName, String tableName, @@ -1354,6 +1371,7 @@ private boolean containsAllPrivilege(Set requestedPrivileges @SafeVarargs private final T alternativeCall( + ClientSupplier clientSupplier, AtomicInteger chosenAlternative, Call... alternatives) throws TException @@ -1363,7 +1381,7 @@ private final T alternativeCall( checkArgument(chosen == Integer.MAX_VALUE || (0 <= chosen && chosen < alternatives.length), "Bad chosen alternative value: %s", chosen); if (chosen != Integer.MAX_VALUE) { - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = clientSupplier.createMetastoreClient()) { return alternatives[chosen].callOn(client); } } @@ -1371,7 +1389,7 @@ private final T alternativeCall( Exception firstException = null; for (int i = 0; i < alternatives.length; i++) { int position = i; - try (ThriftMetastoreClient client = createMetastoreClient()) { + try (ThriftMetastoreClient client = clientSupplier.createMetastoreClient()) { T result = alternatives[i].callOn(client); chosenAlternative.updateAndGet(currentChosen -> Math.min(currentChosen, position)); return result; @@ -1397,6 +1415,36 @@ private ThriftMetastoreClient createMetastoreClient() return clientProvider.createMetastoreClient(); } + private ThriftMetastoreClient createMetastoreClient(HiveIdentity identity) + throws TException + { + ThriftMetastoreClient client = createMetastoreClient(); + if (!impersonationEnabled) { + return client; + } + + setMetastoreUserOrClose(client, identity.getUsername() + .orElseThrow(() -> new IllegalStateException("End-user name should exist when metastore impersonation is enabled"))); + return client; + } + + private static void setMetastoreUserOrClose(ThriftMetastoreClient client, String username) + throws TException + { + try { + client.setUGI(username); + } + catch (Throwable t) { + // close client and suppress any error from close + try (Closeable ignored = client) { + throw t; + } + catch (IOException e) { + // impossible; will be suppressed + } + } + } + private RetryDriver retry() { return RetryDriver.retry() @@ -1414,6 +1462,13 @@ private static RuntimeException propagate(Throwable throwable) throw new RuntimeException(throwable); } + @FunctionalInterface + private interface ClientSupplier + { + ThriftMetastoreClient createMetastoreClient() + throws TException; + } + @FunctionalInterface private interface Call { diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftHiveMetastoreConfig.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftHiveMetastoreConfig.java index 346cbe5f8ee80..df9d130288ed3 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftHiveMetastoreConfig.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftHiveMetastoreConfig.java @@ -31,6 +31,7 @@ public class ThriftHiveMetastoreConfig private Duration minBackoffDelay = RetryDriver.DEFAULT_SLEEP_TIME; private Duration maxBackoffDelay = RetryDriver.DEFAULT_SLEEP_TIME; private Duration maxRetryTime = RetryDriver.DEFAULT_MAX_RETRY_TIME; + private boolean impersonationEnabled; @NotNull public Duration getMetastoreTimeout() @@ -111,4 +112,17 @@ public ThriftHiveMetastoreConfig setMaxBackoffDelay(Duration maxBackoffDelay) this.maxBackoffDelay = maxBackoffDelay; return this; } + + public boolean isImpersonationEnabled() + { + return impersonationEnabled; + } + + @Config("hive.metastore.thrift.impersonation.enabled") + @ConfigDescription("Should end user be impersonated when communicating with metastore") + public ThriftHiveMetastoreConfig setImpersonationEnabled(boolean impersonationEnabled) + { + this.impersonationEnabled = impersonationEnabled; + return this; + } } diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftMetastore.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftMetastore.java index a6c865e86c4b4..924b8ee702a15 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftMetastore.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/metastore/thrift/ThriftMetastore.java @@ -14,6 +14,7 @@ package io.prestosql.plugin.hive.metastore.thrift; import io.prestosql.plugin.hive.PartitionStatistics; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.HivePrincipal; import io.prestosql.plugin.hive.metastore.HivePrivilegeInfo; import io.prestosql.plugin.hive.metastore.PartitionWithStatistics; @@ -38,17 +39,17 @@ public interface ThriftMetastore { - void createDatabase(Database database); + void createDatabase(HiveIdentity identity, Database database); - void dropDatabase(String databaseName); + void dropDatabase(HiveIdentity identity, String databaseName); - void alterDatabase(String databaseName, Database database); + void alterDatabase(HiveIdentity identity, String databaseName, Database database); - void createTable(Table table); + void createTable(HiveIdentity identity, Table table); - void dropTable(String databaseName, String tableName, boolean deleteData); + void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData); - void alterTable(String databaseName, String tableName, Table table); + void alterTable(HiveIdentity identity, String databaseName, String tableName, Table table); List getAllDatabases(); @@ -60,31 +61,31 @@ public interface ThriftMetastore Optional getDatabase(String databaseName); - void addPartitions(String databaseName, String tableName, List partitions); + void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitions); - void dropPartition(String databaseName, String tableName, List parts, boolean deleteData); + void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData); - void alterPartition(String databaseName, String tableName, PartitionWithStatistics partition); + void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partition); - Optional> getPartitionNames(String databaseName, String tableName); + Optional> getPartitionNames(HiveIdentity identity, String databaseName, String tableName); - Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts); + Optional> getPartitionNamesByParts(HiveIdentity identity, String databaseName, String tableName, List parts); - Optional getPartition(String databaseName, String tableName, List partitionValues); + Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues); - List getPartitionsByNames(String databaseName, String tableName, List partitionNames); + List getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames); - Optional
getTable(String databaseName, String tableName); + Optional
getTable(HiveIdentity identity, String databaseName, String tableName); Set getSupportedColumnStatistics(Type type); - PartitionStatistics getTableStatistics(String databaseName, String tableName); + PartitionStatistics getTableStatistics(HiveIdentity identity, String databaseName, String tableName); - Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames); + Map getPartitionStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames); - void updateTableStatistics(String databaseName, String tableName, Function update); + void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, Function update); - void updatePartitionStatistics(String databaseName, String tableName, String partitionName, Function update); + void updatePartitionStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, Function update); void createRole(String role, String grantor); @@ -104,9 +105,11 @@ public interface ThriftMetastore Set listTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal principal); - default Optional> getFields(String databaseName, String tableName) + boolean isImpersonationEnabled(); + + default Optional> getFields(HiveIdentity identity, String databaseName, String tableName) { - Optional
table = getTable(databaseName, tableName); + Optional
table = getTable(identity, databaseName, tableName); if (!table.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } 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 d414ff51d9a5c..20d57230d8a5d 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 @@ -22,6 +22,7 @@ import io.prestosql.plugin.hive.HiveBasicStatistics; import io.prestosql.plugin.hive.HiveBucketProperty; import io.prestosql.plugin.hive.HiveType; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Column; import io.prestosql.plugin.hive.metastore.Database; import io.prestosql.plugin.hive.metastore.HiveColumnStatistics; @@ -287,22 +288,23 @@ public static Stream listEnabledPrincipals(SemiTransactionalHiveM public static Stream listEnabledTablePrivileges(SemiTransactionalHiveMetastore metastore, String databaseName, String tableName, ConnectorIdentity identity) { - return listTablePrivileges(metastore, databaseName, tableName, listEnabledPrincipals(metastore, identity)); + return listTablePrivileges(metastore, new HiveIdentity(identity), databaseName, tableName, listEnabledPrincipals(metastore, identity)); } - public static Stream listApplicableTablePrivileges(SemiTransactionalHiveMetastore metastore, String databaseName, String tableName, String user) + public static Stream listApplicableTablePrivileges(SemiTransactionalHiveMetastore metastore, String databaseName, String tableName, ConnectorIdentity identity) { + String user = identity.getUser(); HivePrincipal userPrincipal = new HivePrincipal(USER, user); Stream principals = Stream.concat( Stream.of(userPrincipal), listApplicableRoles(metastore, userPrincipal) .map(role -> new HivePrincipal(ROLE, role))); - return listTablePrivileges(metastore, databaseName, tableName, principals); + return listTablePrivileges(metastore, new HiveIdentity(identity), databaseName, tableName, principals); } - private static Stream listTablePrivileges(SemiTransactionalHiveMetastore metastore, String databaseName, String tableName, Stream principals) + private static Stream listTablePrivileges(SemiTransactionalHiveMetastore metastore, HiveIdentity identity, String databaseName, String tableName, Stream principals) { - return principals.flatMap(principal -> metastore.listTablePrivileges(databaseName, tableName, principal).stream()); + return principals.flatMap(principal -> metastore.listTablePrivileges(identity, databaseName, tableName, principal).stream()); } public static boolean isRoleEnabled(ConnectorIdentity identity, Function> listRoleGrants, String role) diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/security/LegacyAccessControl.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/security/LegacyAccessControl.java index a829c1173d44c..82fd4d63ed63f 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/security/LegacyAccessControl.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/security/LegacyAccessControl.java @@ -14,6 +14,7 @@ package io.prestosql.plugin.hive.security; import io.prestosql.plugin.hive.HiveTransactionHandle; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.SemiTransactionalHiveMetastore; import io.prestosql.plugin.hive.metastore.Table; import io.prestosql.spi.connector.ColumnMetadata; @@ -104,7 +105,7 @@ public void checkCanDropTable(ConnectorSecurityContext context, SchemaTableName denyDropTable(tableName.toString()); } - Optional
target = metastoreProvider.apply(((HiveTransactionHandle) context.getTransactionHandle())).getTable(tableName.getSchemaName(), tableName.getTableName()); + Optional
target = metastoreProvider.apply(((HiveTransactionHandle) context.getTransactionHandle())).getTable(new HiveIdentity(context.getIdentity()), tableName.getSchemaName(), tableName.getTableName()); if (!target.isPresent()) { denyDropTable(tableName.toString(), "Table not found"); 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 fce1b6e081dc7..ed0388e135e27 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 @@ -448,7 +448,7 @@ private boolean hasGrantOptionForPrivilege(ConnectorSecurityContext context, Pri metastore, tableName.getSchemaName(), tableName.getTableName(), - context.getIdentity().getUser()) + context.getIdentity()) .anyMatch(privilegeInfo -> privilegeInfo.getHivePrivilege().equals(toHivePrivilege(privilege)) && privilegeInfo.isGrantOption()); } diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/security/SqlStandardAccessControlMetadata.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/security/SqlStandardAccessControlMetadata.java index 868135790594b..1b9e2586bd773 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/security/SqlStandardAccessControlMetadata.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/security/SqlStandardAccessControlMetadata.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.HivePrincipal; import io.prestosql.plugin.hive.metastore.HivePrivilegeInfo; import io.prestosql.plugin.hive.metastore.SemiTransactionalHiveMetastore; @@ -124,7 +125,7 @@ public void grantTablePrivileges(ConnectorSession session, SchemaTableName schem .map(privilege -> new HivePrivilegeInfo(toHivePrivilege(privilege), grantOption, new HivePrincipal(USER, session.getUser()), new HivePrincipal(USER, session.getUser()))) .collect(toSet()); - metastore.grantTablePrivileges(schemaName, tableName, grantee, hivePrivilegeInfos); + metastore.grantTablePrivileges(new HiveIdentity(session), schemaName, tableName, grantee, hivePrivilegeInfos); } @Override @@ -137,7 +138,7 @@ public void revokeTablePrivileges(ConnectorSession session, SchemaTableName sche .map(privilege -> new HivePrivilegeInfo(toHivePrivilege(privilege), grantOption, new HivePrincipal(USER, session.getUser()), new HivePrincipal(USER, session.getUser()))) .collect(toSet()); - metastore.revokeTablePrivileges(schemaName, tableName, grantee, hivePrivilegeInfos); + metastore.revokeTablePrivileges(new HiveIdentity(session), schemaName, tableName, grantee, hivePrivilegeInfos); } @Override @@ -149,7 +150,7 @@ public List listTablePrivileges(ConnectorSession session, List result = ImmutableList.builder(); for (SchemaTableName tableName : tableNames) { try { - result.addAll(buildGrants(principals, isAdminRoleSet, tableName)); + result.addAll(buildGrants(session, principals, isAdminRoleSet, tableName)); } catch (TableNotFoundException e) { // table disappeared during listing operation @@ -158,22 +159,22 @@ public List listTablePrivileges(ConnectorSession session, List buildGrants(Set principals, boolean isAdminRoleSet, SchemaTableName tableName) + private List buildGrants(ConnectorSession session, Set principals, boolean isAdminRoleSet, SchemaTableName tableName) { if (isAdminRoleSet) { - return buildGrants(tableName, null); + return buildGrants(session, tableName, null); } ImmutableList.Builder result = ImmutableList.builder(); for (HivePrincipal grantee : principals) { - result.addAll(buildGrants(tableName, grantee)); + result.addAll(buildGrants(session, tableName, grantee)); } return result.build(); } - private List buildGrants(SchemaTableName tableName, HivePrincipal principal) + private List buildGrants(ConnectorSession session, SchemaTableName tableName, HivePrincipal principal) { ImmutableList.Builder result = ImmutableList.builder(); - Set hivePrivileges = metastore.listTablePrivileges(tableName.getSchemaName(), tableName.getTableName(), principal); + Set hivePrivileges = metastore.listTablePrivileges(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName(), principal); for (HivePrivilegeInfo hivePrivilege : hivePrivileges) { Set prestoPrivileges = hivePrivilege.toPrivilegeInfo(); for (PrivilegeInfo prestoPrivilege : prestoPrivileges) { diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/statistics/MetastoreHiveStatisticsProvider.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/statistics/MetastoreHiveStatisticsProvider.java index 9abca323120fb..63fa1e87facff 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/statistics/MetastoreHiveStatisticsProvider.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/statistics/MetastoreHiveStatisticsProvider.java @@ -27,6 +27,7 @@ import io.prestosql.plugin.hive.HiveColumnHandle; import io.prestosql.plugin.hive.HivePartition; import io.prestosql.plugin.hive.PartitionStatistics; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.DateStatistics; import io.prestosql.plugin.hive.metastore.DecimalStatistics; import io.prestosql.plugin.hive.metastore.DoubleStatistics; @@ -100,7 +101,7 @@ public class MetastoreHiveStatisticsProvider public MetastoreHiveStatisticsProvider(SemiTransactionalHiveMetastore metastore) { requireNonNull(metastore, "metastore is null"); - this.statisticsProvider = (table, hivePartitions) -> getPartitionsStatistics(metastore, table, hivePartitions); + this.statisticsProvider = (session, table, hivePartitions) -> getPartitionsStatistics(session, metastore, table, hivePartitions); } @VisibleForTesting @@ -109,7 +110,7 @@ public MetastoreHiveStatisticsProvider(SemiTransactionalHiveMetastore metastore) this.statisticsProvider = requireNonNull(statisticsProvider, "statisticsProvider is null"); } - private static Map getPartitionsStatistics(SemiTransactionalHiveMetastore metastore, SchemaTableName table, List hivePartitions) + private static Map getPartitionsStatistics(ConnectorSession session, SemiTransactionalHiveMetastore metastore, SchemaTableName table, List hivePartitions) { if (hivePartitions.isEmpty()) { return ImmutableMap.of(); @@ -117,12 +118,12 @@ private static Map getPartitionsStatistics(SemiTran boolean unpartitioned = hivePartitions.stream().anyMatch(partition -> partition.getPartitionId().equals(UNPARTITIONED_ID)); if (unpartitioned) { checkArgument(hivePartitions.size() == 1, "expected only one hive partition"); - return ImmutableMap.of(UNPARTITIONED_ID, metastore.getTableStatistics(table.getSchemaName(), table.getTableName())); + return ImmutableMap.of(UNPARTITIONED_ID, metastore.getTableStatistics(new HiveIdentity(session), table.getSchemaName(), table.getTableName())); } Set partitionNames = hivePartitions.stream() .map(HivePartition::getPartitionId) .collect(toImmutableSet()); - return metastore.getPartitionStatistics(table.getSchemaName(), table.getTableName(), partitionNames); + return metastore.getPartitionStatistics(new HiveIdentity(session), table.getSchemaName(), table.getTableName(), partitionNames); } @Override @@ -142,7 +143,7 @@ public TableStatistics getTableStatistics( int sampleSize = getPartitionStatisticsSampleSize(session); List partitionsSample = getPartitionsSample(partitions, sampleSize); try { - Map statisticsSample = statisticsProvider.getPartitionsStatistics(table, partitionsSample); + Map statisticsSample = statisticsProvider.getPartitionsStatistics(session, table, partitionsSample); validatePartitionStatistics(table, statisticsSample); return getTableStatistics(columns, columnTypes, partitions, statisticsSample); } @@ -847,6 +848,6 @@ private static Optional createDecimalRange(DecimalStatistics statis @VisibleForTesting interface PartitionsStatisticsProvider { - Map getPartitionsStatistics(SchemaTableName table, List hivePartitions); + Map getPartitionsStatistics(ConnectorSession session, SchemaTableName table, List hivePartitions); } } diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHive.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHive.java index 7b9d4ef83c95e..68f3a0e929c01 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHive.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHive.java @@ -27,6 +27,7 @@ import io.prestosql.GroupByHashPageIndexerFactory; import io.prestosql.plugin.hive.HdfsEnvironment.HdfsContext; import io.prestosql.plugin.hive.LocationService.WriteInfo; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.authentication.NoHdfsAuthentication; import io.prestosql.plugin.hive.metastore.Column; import io.prestosql.plugin.hive.metastore.HiveColumnStatistics; @@ -1053,7 +1054,7 @@ protected void doTestMismatchSchemaTable( try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(session); - Table oldTable = transaction.getMetastore().getTable(schemaName, tableName).get(); + Table oldTable = transaction.getMetastore().getTable(new HiveIdentity(session), schemaName, tableName).get(); HiveTypeTranslator hiveTypeTranslator = new HiveTypeTranslator(); List dataColumns = tableAfter.stream() .filter(columnMetadata -> !columnMetadata.getName().equals("ds")) @@ -1062,7 +1063,7 @@ protected void doTestMismatchSchemaTable( Table.Builder newTable = Table.builder(oldTable) .setDataColumns(dataColumns); - transaction.getMetastore().replaceTable(schemaName, tableName, newTable.build(), principalPrivileges); + transaction.getMetastore().replaceTable(new HiveIdentity(session), schemaName, tableName, newTable.build(), principalPrivileges); transaction.commit(); } @@ -1981,7 +1982,7 @@ private void assertEmptyFile(HiveStorageFormat format) List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); Table table = transaction.getMetastore() - .getTable(tableName.getSchemaName(), tableName.getTableName()) + .getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(AssertionError::new); // verify directory is empty @@ -2140,7 +2141,7 @@ public void testTableCreationIgnoreExisting() Table table = createSimpleTable(schemaTableName, columns, session, targetPath, "q1"); transaction.getMetastore() .createTable(session, table, privileges, Optional.empty(), false, EMPTY_TABLE_STATISTICS); - Optional
tableHandle = transaction.getMetastore().getTable(schemaName, tableName); + Optional
tableHandle = transaction.getMetastore().getTable(new HiveIdentity(session), schemaName, tableName); assertTrue(tableHandle.isPresent()); transaction.commit(); } @@ -2540,29 +2541,30 @@ public void testUpdateTableColumnStatisticsEmptyOptionalFields() protected void testUpdateTableStatistics(SchemaTableName tableName, PartitionStatistics initialStatistics, PartitionStatistics... statistics) { HiveMetastore metastoreClient = getMetastoreClient(); - assertThat(metastoreClient.getTableStatistics(tableName.getSchemaName(), tableName.getTableName())) + HiveIdentity identity = new HiveIdentity(SESSION); + assertThat(metastoreClient.getTableStatistics(identity, tableName.getSchemaName(), tableName.getTableName())) .isEqualTo(initialStatistics); AtomicReference expectedStatistics = new AtomicReference<>(initialStatistics); for (PartitionStatistics partitionStatistics : statistics) { - metastoreClient.updateTableStatistics(tableName.getSchemaName(), tableName.getTableName(), actualStatistics -> { + metastoreClient.updateTableStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), actualStatistics -> { assertThat(actualStatistics).isEqualTo(expectedStatistics.get()); return partitionStatistics; }); - assertThat(metastoreClient.getTableStatistics(tableName.getSchemaName(), tableName.getTableName())) + assertThat(metastoreClient.getTableStatistics(identity, tableName.getSchemaName(), tableName.getTableName())) .isEqualTo(partitionStatistics); expectedStatistics.set(partitionStatistics); } - assertThat(metastoreClient.getTableStatistics(tableName.getSchemaName(), tableName.getTableName())) + assertThat(metastoreClient.getTableStatistics(identity, tableName.getSchemaName(), tableName.getTableName())) .isEqualTo(expectedStatistics.get()); - metastoreClient.updateTableStatistics(tableName.getSchemaName(), tableName.getTableName(), actualStatistics -> { + metastoreClient.updateTableStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), actualStatistics -> { assertThat(actualStatistics).isEqualTo(expectedStatistics.get()); return initialStatistics; }); - assertThat(metastoreClient.getTableStatistics(tableName.getSchemaName(), tableName.getTableName())) + assertThat(metastoreClient.getTableStatistics(identity, tableName.getSchemaName(), tableName.getTableName())) .isEqualTo(initialStatistics); } @@ -2687,7 +2689,8 @@ protected void createDummyPartitionedTable(SchemaTableName tableName, List new TableNotFoundException(tableName)); List firstPartitionValues = ImmutableList.of("2016-01-01"); @@ -2700,9 +2703,9 @@ protected void createDummyPartitionedTable(SchemaTableName tableName, List new PartitionWithStatistics(createDummyPartition(table, partitionName), partitionName, PartitionStatistics.empty())) .collect(toImmutableList()); - metastoreClient.addPartitions(tableName.getSchemaName(), tableName.getTableName(), partitions); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, currentStatistics -> EMPTY_TABLE_STATISTICS); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, currentStatistics -> EMPTY_TABLE_STATISTICS); + metastoreClient.addPartitions(identity, tableName.getSchemaName(), tableName.getTableName(), partitions); + metastoreClient.updatePartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, currentStatistics -> EMPTY_TABLE_STATISTICS); + metastoreClient.updatePartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, currentStatistics -> EMPTY_TABLE_STATISTICS); } protected void testUpdatePartitionStatistics( @@ -2717,7 +2720,8 @@ protected void testUpdatePartitionStatistics( String secondPartitionName = "ds=2016-01-02"; HiveMetastore metastoreClient = getMetastoreClient(); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) + HiveIdentity identity = new HiveIdentity(SESSION); + assertThat(metastoreClient.getPartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) .isEqualTo(ImmutableMap.of(firstPartitionName, initialStatistics, secondPartitionName, initialStatistics)); AtomicReference expectedStatisticsPartition1 = new AtomicReference<>(initialStatistics); @@ -2726,31 +2730,31 @@ protected void testUpdatePartitionStatistics( for (int i = 0; i < firstPartitionStatistics.size(); i++) { PartitionStatistics statisticsPartition1 = firstPartitionStatistics.get(i); PartitionStatistics statisticsPartition2 = secondPartitionStatistics.get(i); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, actualStatistics -> { + metastoreClient.updatePartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, actualStatistics -> { assertThat(actualStatistics).isEqualTo(expectedStatisticsPartition1.get()); return statisticsPartition1; }); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, actualStatistics -> { + metastoreClient.updatePartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, actualStatistics -> { assertThat(actualStatistics).isEqualTo(expectedStatisticsPartition2.get()); return statisticsPartition2; }); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) + assertThat(metastoreClient.getPartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) .isEqualTo(ImmutableMap.of(firstPartitionName, statisticsPartition1, secondPartitionName, statisticsPartition2)); expectedStatisticsPartition1.set(statisticsPartition1); expectedStatisticsPartition2.set(statisticsPartition2); } - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) + assertThat(metastoreClient.getPartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) .isEqualTo(ImmutableMap.of(firstPartitionName, expectedStatisticsPartition1.get(), secondPartitionName, expectedStatisticsPartition2.get())); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, currentStatistics -> { + metastoreClient.updatePartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, currentStatistics -> { assertThat(currentStatistics).isEqualTo(expectedStatisticsPartition1.get()); return initialStatistics; }); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, currentStatistics -> { + metastoreClient.updatePartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, currentStatistics -> { assertThat(currentStatistics).isEqualTo(expectedStatisticsPartition2.get()); return initialStatistics; }); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) + assertThat(metastoreClient.getPartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) .isEqualTo(ImmutableMap.of(firstPartitionName, initialStatistics, secondPartitionName, initialStatistics)); } @@ -2774,7 +2778,8 @@ protected void testStorePartitionWithStatistics( doCreateEmptyTable(tableName, ORC, columns); HiveMetastore metastoreClient = getMetastoreClient(); - Table table = metastoreClient.getTable(tableName.getSchemaName(), tableName.getTableName()) + HiveIdentity identity = new HiveIdentity(SESSION); + Table table = metastoreClient.getTable(identity, tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); List partitionValues = ImmutableList.of("2016-01-01"); @@ -2783,11 +2788,11 @@ protected void testStorePartitionWithStatistics( Partition partition = createDummyPartition(table, partitionName); // create partition with stats for all columns - metastoreClient.addPartitions(tableName.getSchemaName(), tableName.getTableName(), ImmutableList.of(new PartitionWithStatistics(partition, partitionName, statsForAllColumns1))); + metastoreClient.addPartitions(identity, tableName.getSchemaName(), tableName.getTableName(), ImmutableList.of(new PartitionWithStatistics(partition, partitionName, statsForAllColumns1))); assertEquals( - metastoreClient.getPartition(tableName.getSchemaName(), tableName.getTableName(), partitionValues).get().getStorage().getStorageFormat(), + metastoreClient.getPartition(identity, tableName.getSchemaName(), tableName.getTableName(), partitionValues).get().getStorage().getStorageFormat(), fromHiveStorageFormat(ORC)); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(partitionName))) + assertThat(metastoreClient.getPartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(partitionName))) .isEqualTo(ImmutableMap.of(partitionName, statsForAllColumns1)); // alter the partition into one with other stats @@ -2796,11 +2801,11 @@ protected void testStorePartitionWithStatistics( .setStorageFormat(fromHiveStorageFormat(RCBINARY)) .setLocation(partitionTargetPath(tableName, partitionName))) .build(); - metastoreClient.alterPartition(tableName.getSchemaName(), tableName.getTableName(), new PartitionWithStatistics(modifiedPartition, partitionName, statsForAllColumns2)); + metastoreClient.alterPartition(identity, tableName.getSchemaName(), tableName.getTableName(), new PartitionWithStatistics(modifiedPartition, partitionName, statsForAllColumns2)); assertEquals( - metastoreClient.getPartition(tableName.getSchemaName(), tableName.getTableName(), partitionValues).get().getStorage().getStorageFormat(), + metastoreClient.getPartition(identity, tableName.getSchemaName(), tableName.getTableName(), partitionValues).get().getStorage().getStorageFormat(), fromHiveStorageFormat(RCBINARY)); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(partitionName))) + assertThat(metastoreClient.getPartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(partitionName))) .isEqualTo(ImmutableMap.of(partitionName, statsForAllColumns2)); // alter the partition into one with stats for only subset of columns @@ -2809,8 +2814,8 @@ protected void testStorePartitionWithStatistics( .setStorageFormat(fromHiveStorageFormat(TEXTFILE)) .setLocation(partitionTargetPath(tableName, partitionName))) .build(); - metastoreClient.alterPartition(tableName.getSchemaName(), tableName.getTableName(), new PartitionWithStatistics(modifiedPartition, partitionName, statsForSubsetOfColumns)); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(partitionName))) + metastoreClient.alterPartition(identity, tableName.getSchemaName(), tableName.getTableName(), new PartitionWithStatistics(modifiedPartition, partitionName, statsForSubsetOfColumns)); + assertThat(metastoreClient.getPartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(partitionName))) .isEqualTo(ImmutableMap.of(partitionName, statsForSubsetOfColumns)); // alter the partition into one without stats @@ -2819,8 +2824,8 @@ protected void testStorePartitionWithStatistics( .setStorageFormat(fromHiveStorageFormat(TEXTFILE)) .setLocation(partitionTargetPath(tableName, partitionName))) .build(); - metastoreClient.alterPartition(tableName.getSchemaName(), tableName.getTableName(), new PartitionWithStatistics(modifiedPartition, partitionName, emptyStatistics)); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(partitionName))) + metastoreClient.alterPartition(identity, tableName.getSchemaName(), tableName.getTableName(), new PartitionWithStatistics(modifiedPartition, partitionName, emptyStatistics)); + assertThat(metastoreClient.getPartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(partitionName))) .isEqualTo(ImmutableMap.of(partitionName, emptyStatistics)); } finally { @@ -2850,7 +2855,7 @@ protected String partitionTargetPath(SchemaTableName schemaTableName, String par ConnectorSession session = newSession(); SemiTransactionalHiveMetastore metastore = transaction.getMetastore(); LocationService locationService = getLocationService(); - Table table = metastore.getTable(schemaTableName.getSchemaName(), schemaTableName.getTableName()).get(); + Table table = metastore.getTable(new HiveIdentity(session), schemaTableName.getSchemaName(), schemaTableName.getTableName()).get(); LocationHandle handle = locationService.forExistingTable(metastore, session, table); return locationService.getPartitionWriteInfo(handle, Optional.empty(), partitionName).getTargetPath().toString(); } @@ -2875,8 +2880,9 @@ protected void testPartitionStatisticsSampling(List columns, Par try { createDummyPartitionedTable(tableName, columns); HiveMetastore metastoreClient = getMetastoreClient(); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-01", actualStatistics -> statistics); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-02", actualStatistics -> statistics); + HiveIdentity identity = new HiveIdentity(SESSION); + metastoreClient.updatePartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-01", actualStatistics -> statistics); + metastoreClient.updatePartitionStatistics(identity, tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-02", actualStatistics -> statistics); try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); @@ -3027,12 +3033,12 @@ protected void doCreateTable(SchemaTableName tableName, HiveStorageFormat storag assertEqualsIgnoreOrder(result.getMaterializedRows(), CREATE_TABLE_DATA.getMaterializedRows()); // verify the node version and query ID in table - Table table = getMetastoreClient().getTable(tableName.getSchemaName(), tableName.getTableName()).get(); + Table table = getMetastoreClient().getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()).get(); assertEquals(table.getParameters().get(PRESTO_VERSION_NAME), TEST_SERVER_VERSION); assertEquals(table.getParameters().get(PRESTO_QUERY_ID_NAME), queryId); // verify basic statistics - HiveBasicStatistics statistics = getBasicStatisticsForTable(transaction, tableName); + HiveBasicStatistics statistics = getBasicStatisticsForTable(session, transaction, tableName); assertEquals(statistics.getRowCount().getAsLong(), CREATE_TABLE_DATA.getRowCount()); assertEquals(statistics.getFileCount().getAsLong(), 1L); assertGreaterThan(statistics.getInMemoryDataSizeInBytes().getAsLong(), 0L); @@ -3080,7 +3086,7 @@ protected void doCreateEmptyTable(SchemaTableName tableName, HiveStorageFormat s assertEquals(filterNonHiddenColumnMetadata(tableMetadata.getColumns()), expectedColumns); // verify table format - Table table = transaction.getMetastore().getTable(tableName.getSchemaName(), tableName.getTableName()).get(); + Table table = transaction.getMetastore().getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()).get(); assertEquals(table.getStorage().getStorageFormat().getInputFormat(), storageFormat.getInputFormat()); // verify the node version and query ID @@ -3094,7 +3100,7 @@ protected void doCreateEmptyTable(SchemaTableName tableName, HiveStorageFormat s // verify basic statistics if (partitionedBy.isEmpty()) { - HiveBasicStatistics statistics = getBasicStatisticsForTable(transaction, tableName); + HiveBasicStatistics statistics = getBasicStatisticsForTable(session, transaction, tableName); assertEquals(statistics.getRowCount().getAsLong(), 0L); assertEquals(statistics.getFileCount().getAsLong(), 0L); assertEquals(statistics.getInMemoryDataSizeInBytes().getAsLong(), 0L); @@ -3131,7 +3137,7 @@ private void doInsert(HiveStorageFormat storageFormat, SchemaTableName tableName assertEqualsIgnoreOrder(result.getMaterializedRows(), resultBuilder.build().getMaterializedRows()); // statistics - HiveBasicStatistics tableStatistics = getBasicStatisticsForTable(transaction, tableName); + HiveBasicStatistics tableStatistics = getBasicStatisticsForTable(session, transaction, tableName); assertEquals(tableStatistics.getRowCount().getAsLong(), CREATE_TABLE_DATA.getRowCount() * (i + 1)); assertEquals(tableStatistics.getFileCount().getAsLong(), i + 1L); assertGreaterThan(tableStatistics.getInMemoryDataSizeInBytes().getAsLong(), 0L); @@ -3162,12 +3168,12 @@ private void doInsert(HiveStorageFormat storageFormat, SchemaTableName tableName metadata.finishInsert(session, insertTableHandle, fragments, ImmutableList.of()); // statistics, visible from within transaction - HiveBasicStatistics tableStatistics = getBasicStatisticsForTable(transaction, tableName); + HiveBasicStatistics tableStatistics = getBasicStatisticsForTable(session, transaction, tableName); assertEquals(tableStatistics.getRowCount().getAsLong(), CREATE_TABLE_DATA.getRowCount() * 5L); try (Transaction otherTransaction = newTransaction()) { // statistics, not visible from outside transaction - HiveBasicStatistics otherTableStatistics = getBasicStatisticsForTable(otherTransaction, tableName); + HiveBasicStatistics otherTableStatistics = getBasicStatisticsForTable(session, otherTransaction, tableName); assertEquals(otherTableStatistics.getRowCount().getAsLong(), CREATE_TABLE_DATA.getRowCount() * 3L); } @@ -3207,7 +3213,8 @@ private void doInsert(HiveStorageFormat storageFormat, SchemaTableName tableName // verify statistics unchanged try (Transaction transaction = newTransaction()) { - HiveBasicStatistics statistics = getBasicStatisticsForTable(transaction, tableName); + ConnectorSession session = newSession(); + HiveBasicStatistics statistics = getBasicStatisticsForTable(session, transaction, tableName); assertEquals(statistics.getRowCount().getAsLong(), CREATE_TABLE_DATA.getRowCount() * 3L); assertEquals(statistics.getFileCount().getAsLong(), 3L); } @@ -3250,7 +3257,7 @@ private void doInsertOverwriteUnpartitioned(SchemaTableName tableName) assertEqualsIgnoreOrder(result.getMaterializedRows(), overwriteData.getMaterializedRows()); // statistics - HiveBasicStatistics tableStatistics = getBasicStatisticsForTable(transaction, tableName); + HiveBasicStatistics tableStatistics = getBasicStatisticsForTable(session, transaction, tableName); assertEquals(tableStatistics.getRowCount().getAsLong(), overwriteData.getRowCount()); assertEquals(tableStatistics.getFileCount().getAsLong(), 1L); assertGreaterThan(tableStatistics.getInMemoryDataSizeInBytes().getAsLong(), 0L); @@ -3282,12 +3289,12 @@ private void doInsertOverwriteUnpartitioned(SchemaTableName tableName) metadata.finishInsert(session, insertTableHandle, fragments, ImmutableList.of()); // statistics, visible from within transaction - HiveBasicStatistics tableStatistics = getBasicStatisticsForTable(transaction, tableName); + HiveBasicStatistics tableStatistics = getBasicStatisticsForTable(session, transaction, tableName); assertEquals(tableStatistics.getRowCount().getAsLong(), overwriteData.getRowCount() * 4L); try (Transaction otherTransaction = newTransaction()) { // statistics, not visible from outside transaction - HiveBasicStatistics otherTableStatistics = getBasicStatisticsForTable(otherTransaction, tableName); + HiveBasicStatistics otherTableStatistics = getBasicStatisticsForTable(session, otherTransaction, tableName); assertEquals(otherTableStatistics.getRowCount().getAsLong(), overwriteData.getRowCount()); } @@ -3327,7 +3334,8 @@ private void doInsertOverwriteUnpartitioned(SchemaTableName tableName) // verify statistics unchanged try (Transaction transaction = newTransaction()) { - HiveBasicStatistics statistics = getBasicStatisticsForTable(transaction, tableName); + ConnectorSession session = newSession(); + HiveBasicStatistics statistics = getBasicStatisticsForTable(session, transaction, tableName); assertEquals(statistics.getRowCount().getAsLong(), overwriteData.getRowCount()); assertEquals(statistics.getFileCount().getAsLong(), 1L); } @@ -3364,18 +3372,19 @@ protected Path getTargetPathRoot(ConnectorInsertTableHandle insertTableHandle) protected Set listAllDataFiles(Transaction transaction, String schemaName, String tableName) throws IOException { - HdfsContext context = new HdfsContext(newSession(), schemaName, tableName); + HdfsContext hdfsContext = new HdfsContext(newSession(), schemaName, tableName); + HiveIdentity identity = new HiveIdentity(newSession()); Set existingFiles = new HashSet<>(); - for (String location : listAllDataPaths(transaction.getMetastore(), schemaName, tableName)) { - existingFiles.addAll(listAllDataFiles(context, new Path(location))); + for (String location : listAllDataPaths(identity, transaction.getMetastore(), schemaName, tableName)) { + existingFiles.addAll(listAllDataFiles(hdfsContext, new Path(location))); } return existingFiles; } - public static List listAllDataPaths(SemiTransactionalHiveMetastore metastore, String schemaName, String tableName) + public static List listAllDataPaths(HiveIdentity identity, SemiTransactionalHiveMetastore metastore, String schemaName, String tableName) { ImmutableList.Builder locations = ImmutableList.builder(); - Table table = metastore.getTable(schemaName, tableName).get(); + Table table = metastore.getTable(identity, schemaName, tableName).get(); if (table.getStorage().getLocation() != null) { // For partitioned table, there should be nothing directly under this directory. // But including this location in the set makes the directory content assert more @@ -3383,9 +3392,9 @@ public static List listAllDataPaths(SemiTransactionalHiveMetastore metas locations.add(table.getStorage().getLocation()); } - Optional> partitionNames = metastore.getPartitionNames(schemaName, tableName); + Optional> partitionNames = metastore.getPartitionNames(identity, schemaName, tableName); if (partitionNames.isPresent()) { - metastore.getPartitionsByNames(schemaName, tableName, partitionNames.get()).values().stream() + metastore.getPartitionsByNames(identity, schemaName, tableName, partitionNames.get()).values().stream() .map(Optional::get) .map(partition -> partition.getStorage().getLocation()) .filter(location -> !location.startsWith(table.getStorage().getLocation())) @@ -3428,14 +3437,15 @@ private void doInsertIntoNewPartition(HiveStorageFormat storageFormat, SchemaTab Set existingFiles; try (Transaction transaction = newTransaction()) { // verify partitions were created - List partitionNames = transaction.getMetastore().getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) + HiveIdentity identity = new HiveIdentity(newSession()); + List partitionNames = transaction.getMetastore().getPartitionNames(identity, tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream() .map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1)) .collect(toList())); // verify the node versions in partitions - Map> partitions = getMetastoreClient().getPartitionsByNames(tableName.getSchemaName(), tableName.getTableName(), partitionNames); + Map> partitions = getMetastoreClient().getPartitionsByNames(identity, tableName.getSchemaName(), tableName.getTableName(), partitionNames); assertEquals(partitions.size(), partitionNames.size()); for (String partitionName : partitionNames) { Partition partition = partitions.get(partitionName).get(); @@ -3459,7 +3469,7 @@ private void doInsertIntoNewPartition(HiveStorageFormat storageFormat, SchemaTab // test statistics for (String partitionName : partitionNames) { - HiveBasicStatistics partitionStatistics = getBasicStatisticsForPartition(transaction, tableName, partitionName); + HiveBasicStatistics partitionStatistics = getBasicStatisticsForPartition(session, transaction, tableName, partitionName); assertEquals(partitionStatistics.getRowCount().getAsLong(), 1L); assertEquals(partitionStatistics.getFileCount().getAsLong(), 1L); assertGreaterThan(partitionStatistics.getInMemoryDataSizeInBytes().getAsLong(), 0L); @@ -3550,7 +3560,7 @@ private void doInsertIntoExistingPartition(HiveStorageFormat storageFormat, Sche ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); // verify partitions were created - List partitionNames = transaction.getMetastore().getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) + List partitionNames = transaction.getMetastore().getPartitionNames(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream() .map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1)) @@ -3566,7 +3576,7 @@ private void doInsertIntoExistingPartition(HiveStorageFormat storageFormat, Sche // test statistics for (String partitionName : partitionNames) { - HiveBasicStatistics statistics = getBasicStatisticsForPartition(transaction, tableName, partitionName); + HiveBasicStatistics statistics = getBasicStatisticsForPartition(session, transaction, tableName, partitionName); assertEquals(statistics.getRowCount().getAsLong(), i + 1L); assertEquals(statistics.getFileCount().getAsLong(), i + 1L); assertGreaterThan(statistics.getInMemoryDataSizeInBytes().getAsLong(), 0L); @@ -3605,10 +3615,10 @@ private void doInsertIntoExistingPartition(HiveStorageFormat storageFormat, Sche } // verify statistics are visible from within of the current transaction - List partitionNames = transaction.getMetastore().getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) + List partitionNames = transaction.getMetastore().getPartitionNames(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); for (String partitionName : partitionNames) { - HiveBasicStatistics partitionStatistics = getBasicStatisticsForPartition(transaction, tableName, partitionName); + HiveBasicStatistics partitionStatistics = getBasicStatisticsForPartition(session, transaction, tableName, partitionName); assertEquals(partitionStatistics.getRowCount().getAsLong(), 5L); } @@ -3630,14 +3640,15 @@ private void doInsertIntoExistingPartition(HiveStorageFormat storageFormat, Sche assertEquals(listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName()), existingFiles); // verify temp directory is empty - HdfsContext context = new HdfsContext(session, tableName.getSchemaName(), tableName.getTableName()); - assertTrue(listAllDataFiles(context, stagingPathRoot).isEmpty()); + HdfsContext hdfsContext = new HdfsContext(session, tableName.getSchemaName(), tableName.getTableName()); + assertTrue(listAllDataFiles(hdfsContext, stagingPathRoot).isEmpty()); // verify statistics have been rolled back - List partitionNames = transaction.getMetastore().getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) + HiveIdentity identity = new HiveIdentity(session); + List partitionNames = transaction.getMetastore().getPartitionNames(identity, tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); for (String partitionName : partitionNames) { - HiveBasicStatistics partitionStatistics = getBasicStatisticsForPartition(transaction, tableName, partitionName); + HiveBasicStatistics partitionStatistics = getBasicStatisticsForPartition(session, transaction, tableName, partitionName); assertEquals(partitionStatistics.getRowCount().getAsLong(), 3L); } } @@ -3646,6 +3657,7 @@ private void doInsertIntoExistingPartition(HiveStorageFormat storageFormat, Sche private void doInsertIntoExistingPartitionEmptyStatistics(HiveStorageFormat storageFormat, SchemaTableName tableName) throws Exception { + ConnectorSession session = newSession(); doCreateEmptyTable(tableName, storageFormat, CREATE_TABLE_COLUMNS_PARTITIONED); insertData(tableName, CREATE_TABLE_PARTITIONED_DATA); @@ -3654,11 +3666,11 @@ private void doInsertIntoExistingPartitionEmptyStatistics(HiveStorageFormat stor insertData(tableName, CREATE_TABLE_PARTITIONED_DATA); try (Transaction transaction = newTransaction()) { - List partitionNames = transaction.getMetastore().getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) + List partitionNames = transaction.getMetastore().getPartitionNames(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); for (String partitionName : partitionNames) { - HiveBasicStatistics statistics = getBasicStatisticsForPartition(transaction, tableName, partitionName); + HiveBasicStatistics statistics = getBasicStatisticsForPartition(session, transaction, tableName, partitionName); assertThat(statistics.getRowCount()).isNotPresent(); assertThat(statistics.getInMemoryDataSizeInBytes()).isNotPresent(); // fileCount and rawSize statistics are computed on the fly by the metastore, thus cannot be erased @@ -3666,19 +3678,19 @@ private void doInsertIntoExistingPartitionEmptyStatistics(HiveStorageFormat stor } } - private static HiveBasicStatistics getBasicStatisticsForTable(Transaction transaction, SchemaTableName table) + private static HiveBasicStatistics getBasicStatisticsForTable(ConnectorSession session, Transaction transaction, SchemaTableName table) { return transaction .getMetastore() - .getTableStatistics(table.getSchemaName(), table.getTableName()) + .getTableStatistics(new HiveIdentity(session), table.getSchemaName(), table.getTableName()) .getBasicStatistics(); } - private static HiveBasicStatistics getBasicStatisticsForPartition(Transaction transaction, SchemaTableName table, String partitionName) + private static HiveBasicStatistics getBasicStatisticsForPartition(ConnectorSession session, Transaction transaction, SchemaTableName table, String partitionName) { return transaction .getMetastore() - .getPartitionStatistics(table.getSchemaName(), table.getTableName(), ImmutableSet.of(partitionName)) + .getPartitionStatistics(new HiveIdentity(session), table.getSchemaName(), table.getTableName(), ImmutableSet.of(partitionName)) .get(partitionName) .getBasicStatistics(); } @@ -3686,17 +3698,18 @@ private static HiveBasicStatistics getBasicStatisticsForPartition(Transaction tr private void eraseStatistics(SchemaTableName schemaTableName) { HiveMetastore metastoreClient = getMetastoreClient(); - metastoreClient.updateTableStatistics(schemaTableName.getSchemaName(), schemaTableName.getTableName(), statistics -> new PartitionStatistics(createEmptyStatistics(), ImmutableMap.of())); - Table table = metastoreClient.getTable(schemaTableName.getSchemaName(), schemaTableName.getTableName()) + HiveIdentity identity = new HiveIdentity(SESSION); + metastoreClient.updateTableStatistics(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName(), statistics -> new PartitionStatistics(createEmptyStatistics(), ImmutableMap.of())); + Table table = metastoreClient.getTable(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(schemaTableName)); List partitionColumns = table.getPartitionColumns().stream() .map(Column::getName) .collect(toImmutableList()); if (!table.getPartitionColumns().isEmpty()) { - List partitionNames = metastoreClient.getPartitionNames(schemaTableName.getSchemaName(), schemaTableName.getTableName()) + List partitionNames = metastoreClient.getPartitionNames(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName()) .orElse(ImmutableList.of()); List partitions = metastoreClient - .getPartitionsByNames(schemaTableName.getSchemaName(), schemaTableName.getTableName(), partitionNames) + .getPartitionsByNames(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName(), partitionNames) .entrySet() .stream() .map(Map.Entry::getValue) @@ -3705,6 +3718,7 @@ private void eraseStatistics(SchemaTableName schemaTableName) .collect(toImmutableList()); for (Partition partition : partitions) { metastoreClient.updatePartitionStatistics( + identity, schemaTableName.getSchemaName(), schemaTableName.getTableName(), makePartName(partitionColumns, partition.getValues()), @@ -3774,7 +3788,7 @@ private void doTestMetadataDelete(HiveStorageFormat storageFormat, SchemaTableNa ConnectorMetadata metadata = transaction.getMetadata(); // verify partitions were created - List partitionNames = transaction.getMetastore().getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) + List partitionNames = transaction.getMetastore().getPartitionNames(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream() .map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1)) @@ -4432,11 +4446,11 @@ private void alterBucketProperty(SchemaTableName schemaTableName, Optional table = transaction.getMetastore().getTable(schemaName, tableName); + Optional
table = transaction.getMetastore().getTable(new HiveIdentity(session), schemaName, tableName); Table.Builder tableBuilder = Table.builder(table.get()); tableBuilder.getStorageBuilder().setBucketProperty(bucketProperty); PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(tableOwner, session.getUser()); - transaction.getMetastore().replaceTable(schemaName, tableName, tableBuilder.build(), principalPrivileges); + transaction.getMetastore().replaceTable(new HiveIdentity(session), schemaName, tableName, tableBuilder.build(), principalPrivileges); transaction.commit(); } @@ -4630,7 +4644,7 @@ private void doTestTransactionDeleteInsert( transaction.commit(); if (conflictTrigger.isPresent()) { assertTrue(expectQuerySucceed); - conflictTrigger.get().verifyAndCleanup(tableName); + conflictTrigger.get().verifyAndCleanup(session, tableName); } } catch (TestingRollbackException e) { @@ -4639,7 +4653,7 @@ private void doTestTransactionDeleteInsert( catch (PrestoException e) { assertFalse(expectQuerySucceed); if (conflictTrigger.isPresent()) { - conflictTrigger.get().verifyAndCleanup(tableName); + conflictTrigger.get().verifyAndCleanup(newSession(), tableName); } } } @@ -4654,7 +4668,7 @@ private void doTestTransactionDeleteInsert( try (Transaction transaction = newTransaction()) { // verify partitions List partitionNames = transaction.getMetastore() - .getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) + .getPartitionNames(new HiveIdentity(newSession()), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); assertEqualsIgnoreOrder( partitionNames, @@ -4750,7 +4764,7 @@ protected interface ConflictTrigger void triggerConflict(ConnectorSession session, SchemaTableName tableName, ConnectorInsertTableHandle insertTableHandle, List partitionUpdates) throws IOException; - void verifyAndCleanup(SchemaTableName tableName) + void verifyAndCleanup(ConnectorSession session, SchemaTableName tableName) throws IOException; } @@ -4767,27 +4781,28 @@ public void triggerConflict(ConnectorSession session, SchemaTableName tableName, // This method bypasses transaction interface because this method is inherently hacky and doesn't work well with the transaction abstraction. // Additionally, this method is not part of a test. Its purpose is to set up an environment for another test. HiveMetastore metastoreClient = getMetastoreClient(); - Optional partition = metastoreClient.getPartition(tableName.getSchemaName(), tableName.getTableName(), copyPartitionFrom); + Optional partition = metastoreClient.getPartition(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName(), copyPartitionFrom); conflictPartition = Partition.builder(partition.get()) .setValues(toPartitionValues(partitionNameToConflict)) .build(); metastoreClient.addPartitions( + new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName(), ImmutableList.of(new PartitionWithStatistics(conflictPartition, partitionNameToConflict, PartitionStatistics.empty()))); } @Override - public void verifyAndCleanup(SchemaTableName tableName) + public void verifyAndCleanup(ConnectorSession session, SchemaTableName tableName) { // This method bypasses transaction interface because this method is inherently hacky and doesn't work well with the transaction abstraction. // Additionally, this method is not part of a test. Its purpose is to set up an environment for another test. HiveMetastore metastoreClient = getMetastoreClient(); - Optional actualPartition = metastoreClient.getPartition(tableName.getSchemaName(), tableName.getTableName(), toPartitionValues(partitionNameToConflict)); + Optional actualPartition = metastoreClient.getPartition(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName(), toPartitionValues(partitionNameToConflict)); // Make sure the partition inserted to trigger conflict was not overwritten // Checking storage location is sufficient because implement never uses .../pk1=a/pk2=a2 as the directory for partition [b, b2]. assertEquals(actualPartition.get().getStorage().getLocation(), conflictPartition.getStorage().getLocation()); - metastoreClient.dropPartition(tableName.getSchemaName(), tableName.getTableName(), conflictPartition.getValues(), false); + metastoreClient.dropPartition(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName(), conflictPartition.getValues(), false); } } @@ -4802,11 +4817,11 @@ public void triggerConflict(ConnectorSession session, SchemaTableName tableName, // This method bypasses transaction interface because this method is inherently hacky and doesn't work well with the transaction abstraction. // Additionally, this method is not part of a test. Its purpose is to set up an environment for another test. HiveMetastore metastoreClient = getMetastoreClient(); - metastoreClient.dropPartition(tableName.getSchemaName(), tableName.getTableName(), partitionValueToConflict, false); + metastoreClient.dropPartition(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName(), partitionValueToConflict, false); } @Override - public void verifyAndCleanup(SchemaTableName tableName) + public void verifyAndCleanup(ConnectorSession session, SchemaTableName tableName) { // Do not add back the deleted partition because the implementation is expected to move forward instead of backward when delete fails } @@ -4833,7 +4848,7 @@ public void triggerConflict(ConnectorSession session, SchemaTableName tableName, } @Override - public void verifyAndCleanup(SchemaTableName tableName) + public void verifyAndCleanup(ConnectorSession session, SchemaTableName tableName) throws IOException { assertEquals(listDirectory(context, path), ImmutableList.of()); @@ -4865,7 +4880,7 @@ public void triggerConflict(ConnectorSession session, SchemaTableName tableName, } @Override - public void verifyAndCleanup(SchemaTableName tableName) + public void verifyAndCleanup(ConnectorSession session, SchemaTableName tableName) throws IOException { // The file we added to trigger a conflict was cleaned up because it matches the query prefix. diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHiveFileSystem.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHiveFileSystem.java index 822734016d6fa..2d8f00366a26b 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHiveFileSystem.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHiveFileSystem.java @@ -25,6 +25,7 @@ import io.prestosql.plugin.hive.AbstractTestHive.HiveTransaction; import io.prestosql.plugin.hive.AbstractTestHive.Transaction; import io.prestosql.plugin.hive.HdfsEnvironment.HdfsContext; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.authentication.NoHdfsAuthentication; import io.prestosql.plugin.hive.metastore.Database; import io.prestosql.plugin.hive.metastore.HiveMetastore; @@ -470,32 +471,32 @@ public Optional getDatabase(String databaseName) } @Override - public void createTable(Table table, PrincipalPrivileges privileges) + public void createTable(HiveIdentity identity, Table table, PrincipalPrivileges privileges) { // hack to work around the metastore not being configured for S3 or other FS Table.Builder tableBuilder = Table.builder(table); tableBuilder.getStorageBuilder().setLocation("/"); - super.createTable(tableBuilder.build(), privileges); + super.createTable(identity, tableBuilder.build(), privileges); } @Override - public void dropTable(String databaseName, String tableName, boolean deleteData) + public void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) { try { - Optional
table = getTable(databaseName, tableName); + Optional
table = getTable(identity, databaseName, tableName); if (!table.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } // hack to work around the metastore not being configured for S3 or other FS - List locations = listAllDataPaths(databaseName, tableName); + List locations = listAllDataPaths(identity, databaseName, tableName); Table.Builder tableBuilder = Table.builder(table.get()); tableBuilder.getStorageBuilder().setLocation("/"); // drop table - replaceTable(databaseName, tableName, tableBuilder.build(), new PrincipalPrivileges(ImmutableMultimap.of(), ImmutableMultimap.of())); - delegate.dropTable(databaseName, tableName, false); + replaceTable(identity, databaseName, tableName, tableBuilder.build(), new PrincipalPrivileges(ImmutableMultimap.of(), ImmutableMultimap.of())); + delegate.dropTable(identity, databaseName, tableName, false); // drop data if (deleteData) { @@ -515,7 +516,8 @@ public void dropTable(String databaseName, String tableName, boolean deleteData) public void updateTableLocation(String databaseName, String tableName, String location) { - Optional
table = getTable(databaseName, tableName); + HiveIdentity identity = new HiveIdentity(TESTING_CONTEXT.getIdentity()); + Optional
table = getTable(identity, databaseName, tableName); if (!table.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -524,13 +526,13 @@ public void updateTableLocation(String databaseName, String tableName, String lo tableBuilder.getStorageBuilder().setLocation(location); // NOTE: this clears the permissions - replaceTable(databaseName, tableName, tableBuilder.build(), new PrincipalPrivileges(ImmutableMultimap.of(), ImmutableMultimap.of())); + replaceTable(identity, databaseName, tableName, tableBuilder.build(), new PrincipalPrivileges(ImmutableMultimap.of(), ImmutableMultimap.of())); } - private List listAllDataPaths(String schemaName, String tableName) + private List listAllDataPaths(HiveIdentity identity, String schemaName, String tableName) { ImmutableList.Builder locations = ImmutableList.builder(); - Table table = getTable(schemaName, tableName).get(); + Table table = getTable(identity, schemaName, tableName).get(); if (table.getStorage().getLocation() != null) { // For partitioned table, there should be nothing directly under this directory. // But including this location in the set makes the directory content assert more @@ -538,9 +540,9 @@ private List listAllDataPaths(String schemaName, String tableName) locations.add(table.getStorage().getLocation()); } - Optional> partitionNames = getPartitionNames(schemaName, tableName); + Optional> partitionNames = getPartitionNames(identity, schemaName, tableName); if (partitionNames.isPresent()) { - getPartitionsByNames(schemaName, tableName, partitionNames.get()).values().stream() + getPartitionsByNames(identity, schemaName, tableName, partitionNames.get()).values().stream() .map(Optional::get) .map(partition -> partition.getStorage().getLocation()) .filter(location -> !location.startsWith(table.getStorage().getLocation())) diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHiveLocal.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHiveLocal.java index 4cc2544656b49..8d96733a27539 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHiveLocal.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHiveLocal.java @@ -14,6 +14,7 @@ package io.prestosql.plugin.hive; import com.google.common.io.Files; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Database; import io.prestosql.plugin.hive.metastore.HiveMetastore; import io.prestosql.spi.connector.ConnectorMetadata; @@ -29,12 +30,14 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static io.prestosql.testing.TestingConnectorSession.SESSION; import static java.util.Objects.requireNonNull; public abstract class AbstractTestHiveLocal extends AbstractTestHive { private static final String DEFAULT_TEST_DB_NAME = "test"; + private static final HiveIdentity HIVE_IDENTITY = new HiveIdentity(SESSION); private File tempDir; private String testDbName; @@ -58,11 +61,12 @@ public void initialize() HiveMetastore metastore = createMetastore(tempDir); - metastore.createDatabase(Database.builder() - .setDatabaseName(testDbName) - .setOwnerName("public") - .setOwnerType(PrincipalType.ROLE) - .build()); + metastore.createDatabase(HIVE_IDENTITY, + Database.builder() + .setDatabaseName(testDbName) + .setOwnerName("public") + .setOwnerType(PrincipalType.ROLE) + .build()); HiveConfig hiveConfig = new HiveConfig() .setTimeZone("America/Los_Angeles"); @@ -75,7 +79,7 @@ public void cleanup() throws IOException { try { - getMetastoreClient().dropDatabase(testDbName); + getMetastoreClient().dropDatabase(HIVE_IDENTITY, testDbName); } finally { deleteRecursively(tempDir.toPath(), ALLOW_INSECURE); diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/HiveBenchmarkQueryRunner.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/HiveBenchmarkQueryRunner.java index a8c95ba68b0c2..50d78fd62f49c 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/HiveBenchmarkQueryRunner.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/HiveBenchmarkQueryRunner.java @@ -17,6 +17,7 @@ import com.google.common.io.Files; import io.prestosql.Session; import io.prestosql.benchmark.BenchmarkSuite; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Database; import io.prestosql.plugin.hive.metastore.HiveMetastore; import io.prestosql.plugin.tpch.TpchConnectorFactory; @@ -31,6 +32,7 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.prestosql.plugin.hive.metastore.file.FileHiveMetastore.createTestingFileHiveMetastore; +import static io.prestosql.testing.TestingConnectorSession.SESSION; import static io.prestosql.testing.TestingSession.testSessionBuilder; import static java.util.Objects.requireNonNull; @@ -66,11 +68,14 @@ public static LocalQueryRunner createLocalQueryRunner(File tempDir) // add hive File hiveDir = new File(tempDir, "hive_data"); HiveMetastore metastore = createTestingFileHiveMetastore(hiveDir); - metastore.createDatabase(Database.builder() - .setDatabaseName("tpch") - .setOwnerName("public") - .setOwnerType(PrincipalType.ROLE) - .build()); + + HiveIdentity identity = new HiveIdentity(SESSION); + metastore.createDatabase(identity, + Database.builder() + .setDatabaseName("tpch") + .setOwnerName("public") + .setOwnerType(PrincipalType.ROLE) + .build()); HiveConnectorFactory hiveConnectorFactory = new HiveConnectorFactory( "hive", diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/HiveQueryRunner.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/HiveQueryRunner.java index ef37bd0a6f7fb..0c06b13f197fa 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/HiveQueryRunner.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/HiveQueryRunner.java @@ -21,6 +21,7 @@ import io.airlift.tpch.TpchTable; import io.prestosql.Session; import io.prestosql.metadata.QualifiedObjectName; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.authentication.NoHdfsAuthentication; import io.prestosql.plugin.hive.metastore.Database; import io.prestosql.plugin.hive.metastore.file.FileHiveMetastore; @@ -43,6 +44,7 @@ import static io.airlift.units.Duration.nanosSince; import static io.prestosql.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.prestosql.spi.security.SelectedRole.Type.ROLE; +import static io.prestosql.testing.TestingConnectorSession.SESSION; import static io.prestosql.testing.TestingSession.testSessionBuilder; import static io.prestosql.tests.QueryAssertions.copyTpchTables; import static java.lang.String.format; @@ -126,13 +128,14 @@ public static DistributedQueryRunner createQueryRunner(Iterable> ta queryRunner.createCatalog(HIVE_CATALOG, HIVE_CATALOG, hiveProperties); queryRunner.createCatalog(HIVE_BUCKETED_CATALOG, HIVE_CATALOG, hiveBucketedProperties); + HiveIdentity identity = new HiveIdentity(SESSION); if (!metastore.getDatabase(TPCH_SCHEMA).isPresent()) { - metastore.createDatabase(createDatabaseMetastoreObject(TPCH_SCHEMA)); + metastore.createDatabase(identity, createDatabaseMetastoreObject(TPCH_SCHEMA)); copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, createSession(Optional.empty()), tables); } if (!metastore.getDatabase(TPCH_BUCKETED_SCHEMA).isPresent()) { - metastore.createDatabase(createDatabaseMetastoreObject(TPCH_BUCKETED_SCHEMA)); + metastore.createDatabase(identity, createDatabaseMetastoreObject(TPCH_BUCKETED_SCHEMA)); copyTpchTablesBucketed(queryRunner, "tpch", TINY_SCHEMA_NAME, createBucketedSession(Optional.empty()), tables); } diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/TestHivePageSink.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/TestHivePageSink.java index 502f51b04cee0..aaeae34618fc6 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/TestHivePageSink.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/TestHivePageSink.java @@ -25,6 +25,7 @@ import io.airlift.tpch.TpchColumnType; import io.airlift.tpch.TpchColumnTypes; import io.prestosql.GroupByHashPageIndexerFactory; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.HiveMetastore; import io.prestosql.plugin.hive.metastore.HivePageSinkMetadata; import io.prestosql.spi.Page; @@ -240,12 +241,14 @@ private static ConnectorPageSource createPageSource(HiveTransactionHandle transa private static ConnectorPageSink createPageSink(HiveTransactionHandle transaction, HiveConfig config, HiveMetastore metastore, Path outputPath, HiveWriterStats stats) { + ConnectorSession session = getSession(config); + HiveIdentity identity = new HiveIdentity(session); LocationHandle locationHandle = new LocationHandle(outputPath, outputPath, false, DIRECT_TO_TARGET_NEW_DIRECTORY); HiveOutputTableHandle handle = new HiveOutputTableHandle( SCHEMA_NAME, TABLE_NAME, getColumnHandles(), - new HivePageSinkMetadata(new SchemaTableName(SCHEMA_NAME, TABLE_NAME), metastore.getTable(SCHEMA_NAME, TABLE_NAME), ImmutableMap.of()), + new HivePageSinkMetadata(new SchemaTableName(SCHEMA_NAME, TABLE_NAME), metastore.getTable(identity, SCHEMA_NAME, TABLE_NAME), ImmutableMap.of()), locationHandle, config.getHiveStorageFormat(), config.getHiveStorageFormat(), 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 e32141313ddb0..755fae2220c6c 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 @@ -22,6 +22,7 @@ import io.prestosql.plugin.hive.HiveConfig; import io.prestosql.plugin.hive.HiveType; import io.prestosql.plugin.hive.PartitionStatistics; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege; import io.prestosql.plugin.hive.metastore.SortingColumn.Order; import io.prestosql.spi.security.PrestoPrincipal; @@ -45,6 +46,7 @@ import static io.prestosql.spi.statistics.ColumnStatisticType.MAX_VALUE; import static io.prestosql.spi.statistics.ColumnStatisticType.MIN_VALUE; import static io.prestosql.spi.type.VarcharType.createVarcharType; +import static io.prestosql.testing.TestingConnectorSession.SESSION; import static org.testng.Assert.assertEquals; public class TestRecordingHiveMetastore @@ -100,6 +102,7 @@ public class TestRecordingHiveMetastore OptionalLong.of(8)))); private static final HivePrivilegeInfo PRIVILEGE_INFO = new HivePrivilegeInfo(HivePrivilege.SELECT, true, new HivePrincipal(PrincipalType.USER, "grantor"), new HivePrincipal(PrincipalType.USER, "grantee")); private static final RoleGrant ROLE_GRANT = new RoleGrant(new PrestoPrincipal(USER, "grantee"), "role", true); + private static final HiveIdentity HIVE_CONTEXT = new HiveIdentity(SESSION); @Test public void testRecordingHiveMetastore() @@ -111,7 +114,7 @@ public void testRecordingHiveMetastore() RecordingHiveMetastore recordingHiveMetastore = new RecordingHiveMetastore(new TestingHiveMetastore(), recordingHiveConfig); validateMetadata(recordingHiveMetastore); - recordingHiveMetastore.dropDatabase("other_database"); + recordingHiveMetastore.dropDatabase(HIVE_CONTEXT, "other_database"); recordingHiveMetastore.writeRecording(); HiveConfig replayingHiveConfig = recordingHiveConfig @@ -126,17 +129,17 @@ private void validateMetadata(HiveMetastore hiveMetastore) { assertEquals(hiveMetastore.getDatabase("database"), Optional.of(DATABASE)); assertEquals(hiveMetastore.getAllDatabases(), ImmutableList.of("database")); - assertEquals(hiveMetastore.getTable("database", "table"), Optional.of(TABLE)); + assertEquals(hiveMetastore.getTable(HIVE_CONTEXT, "database", "table"), Optional.of(TABLE)); assertEquals(hiveMetastore.getSupportedColumnStatistics(createVarcharType(123)), ImmutableSet.of(MIN_VALUE, MAX_VALUE)); - assertEquals(hiveMetastore.getTableStatistics("database", "table"), PARTITION_STATISTICS); - assertEquals(hiveMetastore.getPartitionStatistics("database", "table", ImmutableSet.of("value")), ImmutableMap.of("value", PARTITION_STATISTICS)); + assertEquals(hiveMetastore.getTableStatistics(HIVE_CONTEXT, "database", "table"), PARTITION_STATISTICS); + assertEquals(hiveMetastore.getPartitionStatistics(HIVE_CONTEXT, "database", "table", ImmutableSet.of("value")), ImmutableMap.of("value", PARTITION_STATISTICS)); assertEquals(hiveMetastore.getAllTables("database"), ImmutableList.of("table")); assertEquals(hiveMetastore.getTablesWithParameter("database", "param", "value3"), ImmutableList.of("table")); assertEquals(hiveMetastore.getAllViews("database"), ImmutableList.of()); - assertEquals(hiveMetastore.getPartition("database", "table", ImmutableList.of("value")), Optional.of(PARTITION)); - 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.getPartition(HIVE_CONTEXT, "database", "table", ImmutableList.of("value")), Optional.of(PARTITION)); + assertEquals(hiveMetastore.getPartitionNames(HIVE_CONTEXT, "database", "table"), Optional.of(ImmutableList.of("value"))); + assertEquals(hiveMetastore.getPartitionNamesByParts(HIVE_CONTEXT, "database", "table", ImmutableList.of("value")), Optional.of(ImmutableList.of("value"))); + assertEquals(hiveMetastore.getPartitionsByNames(HIVE_CONTEXT, "database", "table", ImmutableList.of("value")), ImmutableMap.of("value", Optional.of(PARTITION))); assertEquals(hiveMetastore.listTablePrivileges("database", "table", "owner", new HivePrincipal(USER, "user")), ImmutableSet.of(PRIVILEGE_INFO)); assertEquals(hiveMetastore.listRoles(), ImmutableSet.of("role")); assertEquals(hiveMetastore.listRoleGrants(new HivePrincipal(USER, "user")), ImmutableSet.of(ROLE_GRANT)); @@ -162,7 +165,7 @@ public List getAllDatabases() } @Override - public Optional
getTable(String databaseName, String tableName) + public Optional
getTable(HiveIdentity identity, String databaseName, String tableName) { if (databaseName.equals("database") && tableName.equals("table")) { return Optional.of(TABLE); @@ -182,7 +185,7 @@ public Set getSupportedColumnStatistics(Type type) } @Override - public PartitionStatistics getTableStatistics(String databaseName, String tableName) + public PartitionStatistics getTableStatistics(HiveIdentity identity, String databaseName, String tableName) { if (databaseName.equals("database") && tableName.equals("table")) { return PARTITION_STATISTICS; @@ -192,7 +195,7 @@ public PartitionStatistics getTableStatistics(String databaseName, String tableN } @Override - public Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames) + public Map getPartitionStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames) { if (databaseName.equals("database") && tableName.equals("table") && partitionNames.contains("value")) { return ImmutableMap.of("value", PARTITION_STATISTICS); @@ -227,13 +230,13 @@ public List getAllViews(String databaseName) } @Override - public void dropDatabase(String databaseName) + public void dropDatabase(HiveIdentity identity, String databaseName) { // noop for test purpose } @Override - public Optional getPartition(String databaseName, String tableName, List partitionValues) + public Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues) { if (databaseName.equals("database") && tableName.equals("table") && partitionValues.equals(ImmutableList.of("value"))) { return Optional.of(PARTITION); @@ -243,7 +246,7 @@ public Optional getPartition(String databaseName, String tableName, L } @Override - public Optional> getPartitionNames(String databaseName, String tableName) + public Optional> getPartitionNames(HiveIdentity identity, String databaseName, String tableName) { if (databaseName.equals("database") && tableName.equals("table")) { return Optional.of(ImmutableList.of("value")); @@ -253,7 +256,7 @@ public Optional> getPartitionNames(String databaseName, String tabl } @Override - public Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) + public Optional> getPartitionNamesByParts(HiveIdentity identity, String databaseName, String tableName, List parts) { if (databaseName.equals("database") && tableName.equals("table") && parts.equals(ImmutableList.of("value"))) { return Optional.of(ImmutableList.of("value")); @@ -263,7 +266,7 @@ public Optional> getPartitionNamesByParts(String databaseName, Stri } @Override - public Map> getPartitionsByNames(String databaseName, String tableName, List partitionNames) + public Map> getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames) { if (databaseName.equals("database") && tableName.equals("table") && partitionNames.contains("value")) { return ImmutableMap.of("value", Optional.of(PARTITION)); 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 8960e5f7d2964..e65afb7e4f32e 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 @@ -15,6 +15,7 @@ import io.prestosql.plugin.hive.HiveType; import io.prestosql.plugin.hive.PartitionStatistics; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.spi.security.RoleGrant; import io.prestosql.spi.statistics.ColumnStatisticType; import io.prestosql.spi.type.Type; @@ -41,7 +42,7 @@ public List getAllDatabases() } @Override - public Optional
getTable(String databaseName, String tableName) + public Optional
getTable(HiveIdentity identity, String databaseName, String tableName) { throw new UnsupportedOperationException(); } @@ -53,25 +54,25 @@ public Set getSupportedColumnStatistics(Type type) } @Override - public PartitionStatistics getTableStatistics(String databaseName, String tableName) + public PartitionStatistics getTableStatistics(HiveIdentity identity, String databaseName, String tableName) { throw new UnsupportedOperationException(); } @Override - public Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames) + public Map getPartitionStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames) { throw new UnsupportedOperationException(); } @Override - public void updateTableStatistics(String databaseName, String tableName, Function update) + public void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, Function update) { throw new UnsupportedOperationException(); } @Override - public void updatePartitionStatistics(String databaseName, String tableName, String partitionName, Function update) + public void updatePartitionStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, Function update) { throw new UnsupportedOperationException(); } @@ -95,109 +96,109 @@ public List getAllViews(String databaseName) } @Override - public void createDatabase(Database database) + public void createDatabase(HiveIdentity identity, Database database) { throw new UnsupportedOperationException(); } @Override - public void dropDatabase(String databaseName) + public void dropDatabase(HiveIdentity identity, String databaseName) { throw new UnsupportedOperationException(); } @Override - public void renameDatabase(String databaseName, String newDatabaseName) + public void renameDatabase(HiveIdentity identity, String databaseName, String newDatabaseName) { throw new UnsupportedOperationException(); } @Override - public void createTable(Table table, PrincipalPrivileges principalPrivileges) + public void createTable(HiveIdentity identity, Table table, PrincipalPrivileges principalPrivileges) { throw new UnsupportedOperationException(); } @Override - public void dropTable(String databaseName, String tableName, boolean deleteData) + public void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) { throw new UnsupportedOperationException(); } @Override - public void replaceTable(String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) + public void replaceTable(HiveIdentity identity, String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) { throw new UnsupportedOperationException(); } @Override - public void renameTable(String databaseName, String tableName, String newDatabaseName, String newTableName) + public void renameTable(HiveIdentity identity, String databaseName, String tableName, String newDatabaseName, String newTableName) { throw new UnsupportedOperationException(); } @Override - public void commentTable(String databaseName, String tableName, Optional comment) + public void commentTable(HiveIdentity identity, String databaseName, String tableName, Optional comment) { throw new UnsupportedOperationException(); } @Override - public void addColumn(String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) + public void addColumn(HiveIdentity identity, String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) { throw new UnsupportedOperationException(); } @Override - public void renameColumn(String databaseName, String tableName, String oldColumnName, String newColumnName) + public void renameColumn(HiveIdentity identity, String databaseName, String tableName, String oldColumnName, String newColumnName) { throw new UnsupportedOperationException(); } @Override - public void dropColumn(String databaseName, String tableName, String columnName) + public void dropColumn(HiveIdentity identity, String databaseName, String tableName, String columnName) { throw new UnsupportedOperationException(); } @Override - public Optional getPartition(String databaseName, String tableName, List partitionValues) + public Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues) { throw new UnsupportedOperationException(); } @Override - public Optional> getPartitionNames(String databaseName, String tableName) + public Optional> getPartitionNames(HiveIdentity identity, String databaseName, String tableName) { throw new UnsupportedOperationException(); } @Override - public Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) + public Optional> getPartitionNamesByParts(HiveIdentity identity, String databaseName, String tableName, List parts) { throw new UnsupportedOperationException(); } @Override - public Map> getPartitionsByNames(String databaseName, String tableName, List partitionNames) + public Map> getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames) { throw new UnsupportedOperationException(); } @Override - public void addPartitions(String databaseName, String tableName, List partitions) + public void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitions) { throw new UnsupportedOperationException(); } @Override - public void dropPartition(String databaseName, String tableName, List parts, boolean deleteData) + public void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData) { throw new UnsupportedOperationException(); } @Override - public void alterPartition(String databaseName, String tableName, PartitionWithStatistics partition) + public void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partition) { throw new UnsupportedOperationException(); } @@ -255,4 +256,10 @@ public Set listRoleGrants(HivePrincipal principal) { throw new UnsupportedOperationException(); } + + @Override + public boolean isImpersonationEnabled() + { + throw new UnsupportedOperationException(); + } } diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/cache/TestCachingHiveMetastore.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/cache/TestCachingHiveMetastore.java index ec3955de329ba..98b70faeaada3 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/cache/TestCachingHiveMetastore.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/cache/TestCachingHiveMetastore.java @@ -17,6 +17,7 @@ import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListeningExecutorService; import io.airlift.units.Duration; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Partition; import io.prestosql.plugin.hive.metastore.thrift.BridgingHiveMetastore; import io.prestosql.plugin.hive.metastore.thrift.MetastoreLocator; @@ -41,6 +42,7 @@ import static io.prestosql.plugin.hive.metastore.thrift.MockThriftMetastoreClient.TEST_PARTITION2; import static io.prestosql.plugin.hive.metastore.thrift.MockThriftMetastoreClient.TEST_ROLES; import static io.prestosql.plugin.hive.metastore.thrift.MockThriftMetastoreClient.TEST_TABLE; +import static io.prestosql.testing.TestingConnectorSession.SESSION; import static java.util.concurrent.Executors.newCachedThreadPool; import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; @@ -51,6 +53,8 @@ @Test(singleThreaded = true) public class TestCachingHiveMetastore { + private static final HiveIdentity IDENTITY = new HiveIdentity(SESSION); + private MockThriftMetastoreClient mockClient; private CachingHiveMetastore metastore; private ThriftMetastoreStats stats; @@ -116,21 +120,21 @@ public void testInvalidDbGetAllTAbles() public void testGetTable() { assertEquals(mockClient.getAccessCount(), 0); - assertNotNull(metastore.getTable(TEST_DATABASE, TEST_TABLE)); + assertNotNull(metastore.getTable(IDENTITY, TEST_DATABASE, TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 1); - assertNotNull(metastore.getTable(TEST_DATABASE, TEST_TABLE)); + assertNotNull(metastore.getTable(IDENTITY, TEST_DATABASE, TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertNotNull(metastore.getTable(TEST_DATABASE, TEST_TABLE)); + assertNotNull(metastore.getTable(IDENTITY, TEST_DATABASE, TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 2); } @Test public void testInvalidDbGetTable() { - assertFalse(metastore.getTable(BAD_DATABASE, TEST_TABLE).isPresent()); + assertFalse(metastore.getTable(IDENTITY, BAD_DATABASE, TEST_TABLE).isPresent()); assertEquals(stats.getGetTable().getThriftExceptions().getTotalCount(), 0); assertEquals(stats.getGetTable().getTotalFailures().getTotalCount(), 0); @@ -141,21 +145,21 @@ public void testGetPartitionNames() { ImmutableList expectedPartitions = ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2); assertEquals(mockClient.getAccessCount(), 0); - assertEquals(metastore.getPartitionNames(TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); + assertEquals(metastore.getPartitionNames(IDENTITY, TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 1); - assertEquals(metastore.getPartitionNames(TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); + assertEquals(metastore.getPartitionNames(IDENTITY, TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertEquals(metastore.getPartitionNames(TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); + assertEquals(metastore.getPartitionNames(IDENTITY, TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 2); } @Test public void testInvalidGetPartitionNames() { - assertEquals(metastore.getPartitionNames(BAD_DATABASE, TEST_TABLE).get(), ImmutableList.of()); + assertEquals(metastore.getPartitionNames(IDENTITY, BAD_DATABASE, TEST_TABLE).get(), ImmutableList.of()); } @Test @@ -165,14 +169,14 @@ public void testGetPartitionNamesByParts() ImmutableList expectedPartitions = ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2); assertEquals(mockClient.getAccessCount(), 0); - assertEquals(metastore.getPartitionNamesByParts(TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); + assertEquals(metastore.getPartitionNamesByParts(IDENTITY, TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 1); - assertEquals(metastore.getPartitionNamesByParts(TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); + assertEquals(metastore.getPartitionNamesByParts(IDENTITY, TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertEquals(metastore.getPartitionNamesByParts(TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); + assertEquals(metastore.getPartitionNamesByParts(IDENTITY, TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 2); } @@ -180,35 +184,35 @@ public void testGetPartitionNamesByParts() public void testInvalidGetPartitionNamesByParts() { ImmutableList parts = ImmutableList.of(); - assertFalse(metastore.getPartitionNamesByParts(BAD_DATABASE, TEST_TABLE, parts).isPresent()); + assertFalse(metastore.getPartitionNamesByParts(IDENTITY, BAD_DATABASE, TEST_TABLE, parts).isPresent()); } @Test public void testGetPartitionsByNames() { assertEquals(mockClient.getAccessCount(), 0); - metastore.getTable(TEST_DATABASE, TEST_TABLE); + metastore.getTable(IDENTITY, TEST_DATABASE, TEST_TABLE); assertEquals(mockClient.getAccessCount(), 1); // Select half of the available partitions and load them into the cache - assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).size(), 1); + assertEquals(metastore.getPartitionsByNames(IDENTITY, TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).size(), 1); assertEquals(mockClient.getAccessCount(), 3); // Now select all of the partitions - assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).size(), 2); + assertEquals(metastore.getPartitionsByNames(IDENTITY, TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).size(), 2); // There should be one more access to fetch the remaining partition assertEquals(mockClient.getAccessCount(), 5); // Now if we fetch any or both of them, they should not hit the client - assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).size(), 1); - assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION2)).size(), 1); - assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).size(), 2); + assertEquals(metastore.getPartitionsByNames(IDENTITY, TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).size(), 1); + assertEquals(metastore.getPartitionsByNames(IDENTITY, TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION2)).size(), 1); + assertEquals(metastore.getPartitionsByNames(IDENTITY, TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).size(), 2); assertEquals(mockClient.getAccessCount(), 5); metastore.flushCache(); // Fetching both should only result in one batched access - assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).size(), 2); + assertEquals(metastore.getPartitionsByNames(IDENTITY, TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).size(), 2); assertEquals(mockClient.getAccessCount(), 7); } @@ -243,7 +247,7 @@ public void testListRoles() @Test public void testInvalidGetPartitionsByNames() { - Map> partitionsByNames = metastore.getPartitionsByNames(BAD_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)); + Map> partitionsByNames = metastore.getPartitionsByNames(IDENTITY, BAD_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)); assertEquals(partitionsByNames.size(), 1); Optional onlyElement = Iterables.getOnlyElement(partitionsByNames.values()); assertFalse(onlyElement.isPresent()); diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java index fce2a7d1fc7c8..42db0c6360d10 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java @@ -19,6 +19,7 @@ import io.prestosql.plugin.hive.PartitionStatistics; import io.prestosql.plugin.hive.SchemaAlreadyExistsException; import io.prestosql.plugin.hive.TableAlreadyExistsException; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.HivePrincipal; import io.prestosql.plugin.hive.metastore.HivePrivilegeInfo; import io.prestosql.plugin.hive.metastore.PartitionWithStatistics; @@ -96,7 +97,7 @@ public InMemoryThriftMetastore(File baseDirectory) } @Override - public synchronized void createDatabase(Database database) + public synchronized void createDatabase(HiveIdentity identity, Database database) { requireNonNull(database, "database is null"); @@ -121,7 +122,7 @@ public synchronized void createDatabase(Database database) } @Override - public synchronized void dropDatabase(String databaseName) + public synchronized void dropDatabase(HiveIdentity identity, String databaseName) { if (!databases.containsKey(databaseName)) { throw new SchemaNotFoundException(databaseName); @@ -133,7 +134,7 @@ public synchronized void dropDatabase(String databaseName) } @Override - public synchronized void alterDatabase(String databaseName, Database newDatabase) + public synchronized void alterDatabase(HiveIdentity identity, String databaseName, Database newDatabase) { String newDatabaseName = newDatabase.getName(); @@ -166,7 +167,7 @@ public synchronized List getAllDatabases() } @Override - public synchronized void createTable(Table table) + public synchronized void createTable(HiveIdentity identity, Table table) { TableType tableType = TableType.valueOf(table.getTableType()); checkArgument(EnumSet.of(MANAGED_TABLE, EXTERNAL_TABLE, VIRTUAL_VIEW).contains(tableType), "Invalid table type: %s", tableType); @@ -200,9 +201,9 @@ public synchronized void createTable(Table table) } @Override - public synchronized void dropTable(String databaseName, String tableName, boolean deleteData) + public synchronized void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) { - List locations = listAllDataPaths(this, databaseName, tableName); + List locations = listAllDataPaths(identity, this, databaseName, tableName); SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); Table table = relations.remove(schemaTableName); @@ -224,10 +225,10 @@ public synchronized void dropTable(String databaseName, String tableName, boolea } } - private static List listAllDataPaths(ThriftMetastore metastore, String schemaName, String tableName) + private static List listAllDataPaths(HiveIdentity identity, ThriftMetastore metastore, String schemaName, String tableName) { ImmutableList.Builder locations = ImmutableList.builder(); - Table table = metastore.getTable(schemaName, tableName).get(); + Table table = metastore.getTable(identity, schemaName, tableName).get(); if (table.getSd().getLocation() != null) { // For unpartitioned table, there should be nothing directly under this directory. // But including this location in the set makes the directory content assert more @@ -235,9 +236,9 @@ private static List listAllDataPaths(ThriftMetastore metastore, String s locations.add(table.getSd().getLocation()); } - Optional> partitionNames = metastore.getPartitionNames(schemaName, tableName); + Optional> partitionNames = metastore.getPartitionNames(identity, schemaName, tableName); if (partitionNames.isPresent()) { - metastore.getPartitionsByNames(schemaName, tableName, partitionNames.get()).stream() + metastore.getPartitionsByNames(identity, schemaName, tableName, partitionNames.get()).stream() .map(partition -> partition.getSd().getLocation()) .filter(location -> !location.startsWith(table.getSd().getLocation())) .forEach(locations::add); @@ -247,7 +248,7 @@ private static List listAllDataPaths(ThriftMetastore metastore, String s } @Override - public synchronized void alterTable(String databaseName, String tableName, Table newTable) + public synchronized void alterTable(HiveIdentity identity, String databaseName, String tableName, Table newTable) { SchemaTableName oldName = new SchemaTableName(databaseName, tableName); SchemaTableName newName = new SchemaTableName(newTable.getDbName(), newTable.getTableName()); @@ -316,7 +317,7 @@ public synchronized Optional getDatabase(String databaseName) } @Override - public synchronized void addPartitions(String databaseName, String tableName, List partitionsWithStatistics) + public synchronized void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitionsWithStatistics) { for (PartitionWithStatistics partitionWithStatistics : partitionsWithStatistics) { Partition partition = toMetastoreApiPartition(partitionWithStatistics.getPartition()); @@ -330,14 +331,14 @@ public synchronized void addPartitions(String databaseName, String tableName, Li } @Override - public synchronized void dropPartition(String databaseName, String tableName, List parts, boolean deleteData) + public synchronized void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData) { partitions.entrySet().removeIf(entry -> entry.getKey().matches(databaseName, tableName) && entry.getValue().getValues().equals(parts)); } @Override - public synchronized void alterPartition(String databaseName, String tableName, PartitionWithStatistics partitionWithStatistics) + public synchronized void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partitionWithStatistics) { Partition partition = toMetastoreApiPartition(partitionWithStatistics.getPartition()); if (partition.getParameters() == null) { @@ -349,7 +350,7 @@ public synchronized void alterPartition(String databaseName, String tableName, P } @Override - public synchronized Optional> getPartitionNames(String databaseName, String tableName) + public synchronized Optional> getPartitionNames(HiveIdentity identity, String databaseName, String tableName) { return Optional.of(ImmutableList.copyOf(partitions.entrySet().stream() .filter(entry -> entry.getKey().matches(databaseName, tableName)) @@ -358,7 +359,7 @@ public synchronized Optional> getPartitionNames(String databaseName } @Override - public synchronized Optional getPartition(String databaseName, String tableName, List partitionValues) + public synchronized Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues) { PartitionName name = PartitionName.partition(databaseName, tableName, partitionValues); Partition partition = partitions.get(name); @@ -369,7 +370,7 @@ public synchronized Optional getPartition(String databaseName, String } @Override - public synchronized Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) + public synchronized Optional> getPartitionNamesByParts(HiveIdentity identity, String databaseName, String tableName, List parts) { return Optional.of(partitions.entrySet().stream() .filter(entry -> partitionMatches(entry.getValue(), databaseName, tableName, parts)) @@ -397,7 +398,7 @@ private static boolean partitionMatches(Partition partition, String databaseName } @Override - public synchronized List getPartitionsByNames(String databaseName, String tableName, List partitionNames) + public synchronized List getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames) { ImmutableList.Builder builder = ImmutableList.builder(); for (String name : partitionNames) { @@ -412,7 +413,7 @@ public synchronized List getPartitionsByNames(String databaseName, St } @Override - public synchronized Optional
getTable(String databaseName, String tableName) + public synchronized Optional
getTable(HiveIdentity identity, String databaseName, String tableName) { SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); return Optional.ofNullable(relations.get(schemaTableName)); @@ -425,7 +426,7 @@ public Set getSupportedColumnStatistics(Type type) } @Override - public synchronized PartitionStatistics getTableStatistics(String databaseName, String tableName) + public synchronized PartitionStatistics getTableStatistics(HiveIdentity identity, String databaseName, String tableName) { SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); PartitionStatistics statistics = columnStatistics.get(schemaTableName); @@ -436,7 +437,7 @@ public synchronized PartitionStatistics getTableStatistics(String databaseName, } @Override - public synchronized Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames) + public synchronized Map getPartitionStatistics(HiveIdentity identity, String databaseName, String tableName, Set partitionNames) { ImmutableMap.Builder result = ImmutableMap.builder(); for (String partitionName : partitionNames) { @@ -451,16 +452,16 @@ public synchronized Map getPartitionStatistics(Stri } @Override - public synchronized void updateTableStatistics(String databaseName, String tableName, Function update) + public synchronized void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, Function update) { - columnStatistics.put(new SchemaTableName(databaseName, tableName), update.apply(getTableStatistics(databaseName, tableName))); + columnStatistics.put(new SchemaTableName(databaseName, tableName), update.apply(getTableStatistics(identity, databaseName, tableName))); } @Override - public synchronized void updatePartitionStatistics(String databaseName, String tableName, String partitionName, Function update) + public synchronized void updatePartitionStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, Function update) { PartitionName partitionKey = PartitionName.partition(databaseName, tableName, partitionName); - partitionColumnStatistics.put(partitionKey, update.apply(getPartitionStatistics(databaseName, tableName, ImmutableSet.of(partitionName)).get(partitionName))); + partitionColumnStatistics.put(partitionKey, update.apply(getPartitionStatistics(identity, databaseName, tableName, ImmutableSet.of(partitionName)).get(partitionName))); } @Override @@ -517,6 +518,12 @@ public void revokeTablePrivileges(String databaseName, String tableName, String throw new UnsupportedOperationException(); } + @Override + public boolean isImpersonationEnabled() + { + return false; + } + private static boolean isParentDir(File directory, File baseDirectory) { for (File parent = directory.getParentFile(); parent != null; parent = parent.getParentFile()) { diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/TestThriftHiveMetastoreConfig.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/TestThriftHiveMetastoreConfig.java index 058173e5cc03d..a7696c0e8ef85 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/TestThriftHiveMetastoreConfig.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/metastore/thrift/TestThriftHiveMetastoreConfig.java @@ -36,7 +36,8 @@ public void testDefaults() .setBackoffScaleFactor(2.0) .setMinBackoffDelay(new Duration(1, SECONDS)) .setMaxBackoffDelay(new Duration(1, SECONDS)) - .setMaxRetryTime(new Duration(30, SECONDS))); + .setMaxRetryTime(new Duration(30, SECONDS)) + .setImpersonationEnabled(false)); } @Test @@ -49,6 +50,7 @@ public void testExplicitPropertyMappings() .put("hive.metastore.thrift.client.min-backoff-delay", "2s") .put("hive.metastore.thrift.client.max-backoff-delay", "4s") .put("hive.metastore.thrift.client.max-retry-time", "60s") + .put("hive.metastore.thrift.impersonation.enabled", "true") .build(); ThriftHiveMetastoreConfig expected = new ThriftHiveMetastoreConfig() @@ -57,7 +59,8 @@ public void testExplicitPropertyMappings() .setBackoffScaleFactor(3.0) .setMinBackoffDelay(new Duration(2, SECONDS)) .setMaxBackoffDelay(new Duration(4, SECONDS)) - .setMaxRetryTime(new Duration(60, SECONDS)); + .setMaxRetryTime(new Duration(60, SECONDS)) + .setImpersonationEnabled(true); assertFullMapping(properties, expected); } diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/statistics/TestMetastoreHiveStatisticsProvider.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/statistics/TestMetastoreHiveStatisticsProvider.java index 04485f73b6ed5..454aa25e80357 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/statistics/TestMetastoreHiveStatisticsProvider.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/statistics/TestMetastoreHiveStatisticsProvider.java @@ -609,7 +609,7 @@ public void testGetTableStatistics() .setBasicStatistics(new HiveBasicStatistics(OptionalLong.empty(), OptionalLong.of(1000), OptionalLong.empty(), OptionalLong.empty())) .setColumnStatistics(ImmutableMap.of(COLUMN, createIntegerColumnStatistics(OptionalLong.of(-100), OptionalLong.of(100), OptionalLong.of(500), OptionalLong.of(300)))) .build(); - MetastoreHiveStatisticsProvider statisticsProvider = new MetastoreHiveStatisticsProvider((table, hivePartitions) -> ImmutableMap.of(partitionName, statistics)); + MetastoreHiveStatisticsProvider statisticsProvider = new MetastoreHiveStatisticsProvider((session, table, hivePartitions) -> ImmutableMap.of(partitionName, statistics)); TestingConnectorSession session = new TestingConnectorSession(new HiveSessionProperties(new HiveConfig(), new OrcFileWriterConfig(), new ParquetFileWriterConfig()).getSessionProperties()); HiveColumnHandle columnHandle = new HiveColumnHandle(COLUMN, HIVE_LONG, BIGINT.getTypeSignature(), 2, REGULAR, Optional.empty()); TableStatistics expected = TableStatistics.builder() @@ -659,7 +659,7 @@ public void testGetTableStatisticsUnpartitioned() .setBasicStatistics(new HiveBasicStatistics(OptionalLong.empty(), OptionalLong.of(1000), OptionalLong.empty(), OptionalLong.empty())) .setColumnStatistics(ImmutableMap.of(COLUMN, createIntegerColumnStatistics(OptionalLong.of(-100), OptionalLong.of(100), OptionalLong.of(500), OptionalLong.of(300)))) .build(); - MetastoreHiveStatisticsProvider statisticsProvider = new MetastoreHiveStatisticsProvider((table, hivePartitions) -> ImmutableMap.of(UNPARTITIONED_ID, statistics)); + MetastoreHiveStatisticsProvider statisticsProvider = new MetastoreHiveStatisticsProvider((session, table, hivePartitions) -> ImmutableMap.of(UNPARTITIONED_ID, statistics)); TestingConnectorSession session = new TestingConnectorSession(new HiveSessionProperties(new HiveConfig(), new OrcFileWriterConfig(), new ParquetFileWriterConfig()).getSessionProperties()); HiveColumnHandle columnHandle = new HiveColumnHandle(COLUMN, HIVE_LONG, BIGINT.getTypeSignature(), 2, REGULAR, Optional.empty()); TableStatistics expected = TableStatistics.builder() @@ -686,7 +686,7 @@ public void testGetTableStatisticsUnpartitioned() public void testGetTableStatisticsEmpty() { String partitionName = "p1=string1/p2=1234"; - MetastoreHiveStatisticsProvider statisticsProvider = new MetastoreHiveStatisticsProvider((table, hivePartitions) -> ImmutableMap.of(partitionName, PartitionStatistics.empty())); + MetastoreHiveStatisticsProvider statisticsProvider = new MetastoreHiveStatisticsProvider((session, table, hivePartitions) -> ImmutableMap.of(partitionName, PartitionStatistics.empty())); TestingConnectorSession session = new TestingConnectorSession(new HiveSessionProperties(new HiveConfig(), new OrcFileWriterConfig(), new ParquetFileWriterConfig()).getSessionProperties()); assertEquals( statisticsProvider.getTableStatistics( @@ -701,7 +701,7 @@ public void testGetTableStatisticsEmpty() @Test public void testGetTableStatisticsSampling() { - MetastoreHiveStatisticsProvider statisticsProvider = new MetastoreHiveStatisticsProvider((table, hivePartitions) -> { + MetastoreHiveStatisticsProvider statisticsProvider = new MetastoreHiveStatisticsProvider((session, table, hivePartitions) -> { assertEquals(table, TABLE); assertEquals(hivePartitions.size(), 1); return ImmutableMap.of(); @@ -726,7 +726,7 @@ public void testGetTableStatisticsValidationFailure() .setBasicStatistics(new HiveBasicStatistics(-1, 0, 0, 0)) .build(); String partitionName = "p1=string1/p2=1234"; - MetastoreHiveStatisticsProvider statisticsProvider = new MetastoreHiveStatisticsProvider((table, hivePartitions) -> ImmutableMap.of(partitionName, corruptedStatistics)); + MetastoreHiveStatisticsProvider statisticsProvider = new MetastoreHiveStatisticsProvider((session, table, hivePartitions) -> ImmutableMap.of(partitionName, corruptedStatistics)); TestingConnectorSession session = new TestingConnectorSession(new HiveSessionProperties( new HiveConfig().setIgnoreCorruptedStatistics(false), new OrcFileWriterConfig(), diff --git a/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/HiveTableOperations.java b/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/HiveTableOperations.java index 5b0db97fb5ced..95ab5f293e68f 100644 --- a/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/HiveTableOperations.java +++ b/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/HiveTableOperations.java @@ -17,6 +17,7 @@ import io.prestosql.plugin.hive.HdfsEnvironment; import io.prestosql.plugin.hive.HdfsEnvironment.HdfsContext; import io.prestosql.plugin.hive.HiveType; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Column; import io.prestosql.plugin.hive.metastore.HiveMetastore; import io.prestosql.plugin.hive.metastore.PrincipalPrivileges; @@ -25,6 +26,7 @@ import io.prestosql.spi.PrestoException; import io.prestosql.spi.connector.SchemaTableName; import io.prestosql.spi.connector.TableNotFoundException; +import io.prestosql.spi.security.ConnectorIdentity; import org.apache.hadoop.hive.metastore.TableType; import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; import org.apache.hadoop.mapred.FileInputFormat; @@ -81,6 +83,7 @@ public class HiveTableOperations FileOutputFormat.class.getName()); private final HiveMetastore metastore; + private final HiveIdentity identity; private final String database; private final String tableName; private final Optional owner; @@ -92,25 +95,27 @@ public class HiveTableOperations private boolean shouldRefresh = true; private int version = -1; - public HiveTableOperations(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, HdfsContext hdfsContext, String database, String table) + public HiveTableOperations(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, HdfsContext hdfsContext, HiveIdentity identity, String database, String table) { - this(new HdfsFileIo(hdfsEnvironment, hdfsContext), metastore, database, table, Optional.empty(), Optional.empty()); + this(new HdfsFileIo(hdfsEnvironment, hdfsContext), metastore, identity, database, table, Optional.empty(), Optional.empty()); } - public HiveTableOperations(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, HdfsContext hdfsContext, String database, String table, String owner, String location) + public HiveTableOperations(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, HdfsContext hdfsContext, HiveIdentity identity, String database, String table, String owner, String location) { this(new HdfsFileIo(hdfsEnvironment, hdfsContext), metastore, + identity, database, table, Optional.of(requireNonNull(owner, "owner is null")), Optional.of(requireNonNull(location, "location is null"))); } - private HiveTableOperations(FileIO fileIo, HiveMetastore metastore, String database, String table, Optional owner, Optional location) + private HiveTableOperations(FileIO fileIo, HiveMetastore metastore, HiveIdentity identity, String database, String table, Optional owner, Optional location) { this.fileIo = requireNonNull(fileIo, "fileIo is null"); this.metastore = requireNonNull(metastore, "metastore is null"); + this.identity = requireNonNull(identity, "identity is null"); this.database = requireNonNull(database, "database is null"); this.tableName = requireNonNull(table, "table is null"); this.owner = requireNonNull(owner, "owner is null"); @@ -214,11 +219,13 @@ public void commit(@Nullable TableMetadata base, TableMetadata metadata) } PrincipalPrivileges privileges = buildInitialPrivilegeSet(table.getOwner()); + ConnectorIdentity identity = new ConnectorIdentity(table.getOwner(), Optional.empty(), Optional.empty()); + HiveIdentity context = new HiveIdentity(identity); if (base == null) { - metastore.createTable(table, privileges); + metastore.createTable(context, table, privileges); } else { - metastore.replaceTable(database, tableName, table, privileges); + metastore.replaceTable(context, database, tableName, table, privileges); } shouldRefresh = true; @@ -257,7 +264,7 @@ public LocationProvider locationProvider() private Table getTable() { - return metastore.getTable(database, tableName) + return metastore.getTable(identity, database, tableName) .orElseThrow(() -> new TableNotFoundException(getSchemaTableName())); } diff --git a/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/IcebergMetadata.java b/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/IcebergMetadata.java index cf251a1f3a4a8..25e64ddb6861a 100644 --- a/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/IcebergMetadata.java +++ b/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/IcebergMetadata.java @@ -23,6 +23,7 @@ import io.prestosql.plugin.hive.HiveColumnHandle; import io.prestosql.plugin.hive.HiveWrittenPartitions; import io.prestosql.plugin.hive.TableAlreadyExistsException; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.Database; import io.prestosql.plugin.hive.metastore.HiveMetastore; import io.prestosql.plugin.hive.metastore.Table; @@ -138,7 +139,7 @@ public List listSchemaNames(ConnectorSession session) public IcebergTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) { IcebergTableHandle handle = IcebergTableHandle.from(tableName); - Optional
table = metastore.getTable(handle.getSchemaName(), handle.getTableName()); + Optional
table = metastore.getTable(new HiveIdentity(session), handle.getSchemaName(), handle.getTableName()); if (!table.isPresent()) { return null; } @@ -243,7 +244,7 @@ public void createSchema(ConnectorSession session, String schemaName, Map new SchemaNotFoundException(schemaName)); HdfsContext hdfsContext = new HdfsContext(session, schemaName, tableName); + HiveIdentity identity = new HiveIdentity(session); String targetPath = getTableDefaultLocation(database, hdfsContext, hdfsEnvironment, schemaName, tableName).toString(); - TableOperations operations = new HiveTableOperations(metastore, hdfsEnvironment, hdfsContext, schemaName, tableName, session.getUser(), targetPath); + TableOperations operations = new HiveTableOperations(metastore, hdfsEnvironment, hdfsContext, identity, schemaName, tableName, session.getUser(), targetPath); if (operations.current() != null) { throw new TableAlreadyExistsException(schemaTableName); } @@ -393,14 +395,14 @@ public Optional getInfo(ConnectorTableHandle tableHandle) public void dropTable(ConnectorSession session, ConnectorTableHandle tableHandle) { IcebergTableHandle handle = (IcebergTableHandle) tableHandle; - metastore.dropTable(handle.getSchemaName(), handle.getTableName(), true); + metastore.dropTable(new HiveIdentity(session), handle.getSchemaName(), handle.getTableName(), true); } @Override public void renameTable(ConnectorSession session, ConnectorTableHandle tableHandle, SchemaTableName newTable) { IcebergTableHandle handle = (IcebergTableHandle) tableHandle; - metastore.renameTable(handle.getSchemaName(), handle.getTableName(), newTable.getSchemaName(), newTable.getTableName()); + metastore.renameTable(new HiveIdentity(session), handle.getSchemaName(), handle.getTableName(), newTable.getSchemaName(), newTable.getTableName()); } @Override @@ -431,7 +433,7 @@ public void renameColumn(ConnectorSession session, ConnectorTableHandle tableHan private ConnectorTableMetadata getTableMetadata(ConnectorSession session, SchemaTableName table) { - if (!metastore.getTable(table.getSchemaName(), table.getTableName()).isPresent()) { + if (!metastore.getTable(new HiveIdentity(session), table.getSchemaName(), table.getTableName()).isPresent()) { throw new TableNotFoundException(table); } diff --git a/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/IcebergUtil.java b/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/IcebergUtil.java index 3e5160027b84c..28e7f35fd4e2e 100644 --- a/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/IcebergUtil.java +++ b/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/IcebergUtil.java @@ -22,6 +22,7 @@ import io.prestosql.plugin.hive.HiveType; import io.prestosql.plugin.hive.HiveTypeTranslator; import io.prestosql.plugin.hive.TypeTranslator; +import io.prestosql.plugin.hive.authentication.HiveIdentity; import io.prestosql.plugin.hive.metastore.HiveMetastore; import io.prestosql.spi.connector.ConnectorSession; import io.prestosql.spi.connector.SchemaTableName; @@ -70,8 +71,9 @@ public static boolean isIcebergTable(io.prestosql.plugin.hive.metastore.Table ta public static Table getIcebergTable(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, ConnectorSession session, SchemaTableName table) { - HdfsContext context = new HdfsContext(session, table.getSchemaName(), table.getTableName()); - TableOperations operations = new HiveTableOperations(metastore, hdfsEnvironment, context, table.getSchemaName(), table.getTableName()); + HdfsContext hdfsContext = new HdfsContext(session, table.getSchemaName(), table.getTableName()); + HiveIdentity identity = new HiveIdentity(session); + TableOperations operations = new HiveTableOperations(metastore, hdfsEnvironment, hdfsContext, identity, table.getSchemaName(), table.getTableName()); return new BaseTable(operations, table.getSchemaName() + "." + table.getTableName()); } diff --git a/presto-product-tests/bin/product-tests-suite-1.sh b/presto-product-tests/bin/product-tests-suite-1.sh index 6ca2a26e935f6..8b765ca9e733d 100755 --- a/presto-product-tests/bin/product-tests-suite-1.sh +++ b/presto-product-tests/bin/product-tests-suite-1.sh @@ -4,10 +4,9 @@ set -xeuo pipefail exit_code=0 -# TODO enable avro_schema_url when adding Metastore impersonation presto-product-tests/bin/run_on_docker.sh \ multinode \ - -x quarantine,big_query,storage_formats,profile_specific_tests,tpcds,cassandra,mysql_connector,postgresql_connector,mysql,kafka,avro_schema_url,simba_jdbc \ + -x quarantine,big_query,storage_formats,profile_specific_tests,tpcds,cassandra,mysql_connector,postgresql_connector,mysql,kafka,simba_jdbc \ || exit_code=1 exit "${exit_code}" diff --git a/presto-product-tests/bin/product-tests-suite-3.sh b/presto-product-tests/bin/product-tests-suite-3.sh index 4d20313a10c4c..cd6aa68a3afde 100755 --- a/presto-product-tests/bin/product-tests-suite-3.sh +++ b/presto-product-tests/bin/product-tests-suite-3.sh @@ -29,11 +29,9 @@ presto-product-tests/bin/run_on_docker.sh \ -g storage_formats \ || exit_code=1 -# TODO enable avro_schema_url when adding Metastore impersonation presto-product-tests/bin/run_on_docker.sh \ singlenode-kerberos-kms-hdfs-impersonation \ -g storage_formats \ - -x avro_schema_url \ || exit_code=1 exit "${exit_code}" diff --git a/presto-product-tests/bin/product-tests-suite-5.sh b/presto-product-tests/bin/product-tests-suite-5.sh new file mode 100755 index 0000000000000..15ca5f56470fb --- /dev/null +++ b/presto-product-tests/bin/product-tests-suite-5.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +set -xeuo pipefail + +exit_code=0 + +presto-product-tests/bin/run_on_docker.sh \ + singlenode-hive-impersonation \ + -g storage_formats,hdfs_impersonation \ + || exit_code=1 + +presto-product-tests/bin/run_on_docker.sh \ + singlenode-kerberos-hive-impersonation \ + -g storage_formats,hdfs_impersonation,authorization \ + || exit_code=1 + +exit "${exit_code}" diff --git a/presto-product-tests/conf/docker/singlenode-hive-impersonation/compose.sh b/presto-product-tests/conf/docker/singlenode-hive-impersonation/compose.sh new file mode 100755 index 0000000000000..9d94d229f79eb --- /dev/null +++ b/presto-product-tests/conf/docker/singlenode-hive-impersonation/compose.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +set -euo pipefail + +source "${BASH_SOURCE%/*}/../common/compose-commons.sh" + +docker-compose \ + -f ${BASH_SOURCE%/*}/../common/standard.yml \ + -f ${BASH_SOURCE%/*}/docker-compose.yml \ + "$@" diff --git a/presto-product-tests/conf/docker/singlenode-hive-impersonation/docker-compose.yml b/presto-product-tests/conf/docker/singlenode-hive-impersonation/docker-compose.yml new file mode 100644 index 0000000000000..6e153f881cd34 --- /dev/null +++ b/presto-product-tests/conf/docker/singlenode-hive-impersonation/docker-compose.yml @@ -0,0 +1,6 @@ +version: '2' +services: + + presto-master: + volumes: + - ../../../conf/presto/etc/environment-specific-catalogs/singlenode-hive-impersonation/hive.properties:/docker/volumes/conf/presto/etc/catalog/hive.properties diff --git a/presto-product-tests/conf/docker/singlenode-kerberos-hive-impersonation/compose.sh b/presto-product-tests/conf/docker/singlenode-kerberos-hive-impersonation/compose.sh new file mode 100755 index 0000000000000..ded20a6688252 --- /dev/null +++ b/presto-product-tests/conf/docker/singlenode-kerberos-hive-impersonation/compose.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +set -euo pipefail + +source "${BASH_SOURCE%/*}/../common/compose-commons.sh" + +docker-compose \ + -f ${BASH_SOURCE%/*}/../common/standard.yml \ + -f ${BASH_SOURCE%/*}/../common/kerberos.yml \ + -f ${BASH_SOURCE%/*}/docker-compose.yml \ + "$@" diff --git a/presto-product-tests/conf/docker/singlenode-kerberos-hive-impersonation/docker-compose.yml b/presto-product-tests/conf/docker/singlenode-kerberos-hive-impersonation/docker-compose.yml new file mode 100644 index 0000000000000..3c705b78fb0b4 --- /dev/null +++ b/presto-product-tests/conf/docker/singlenode-kerberos-hive-impersonation/docker-compose.yml @@ -0,0 +1,6 @@ +version: '2' +services: + + presto-master: + volumes: + - ../../../conf/presto/etc/environment-specific-catalogs/singlenode-kerberos-hive-impersonation/hive.properties:/docker/volumes/conf/presto/etc/catalog/hive.properties diff --git a/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-hive-impersonation/hive.properties b/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-hive-impersonation/hive.properties new file mode 100644 index 0000000000000..250706aad1dd8 --- /dev/null +++ b/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-hive-impersonation/hive.properties @@ -0,0 +1,23 @@ +# +# WARNING +# ^^^^^^^ +# This configuration file is for development only and should NOT be used +# in production. For example configuration, see the Presto documentation. +# + +connector.name=hive-hadoop2 +hive.metastore.uri=thrift://hadoop-master:9083 +hive.metastore.thrift.client.socks-proxy=hadoop-master:1180 +hive.config.resources=/docker/volumes/conf/presto/etc/hive-default-fs-site.xml +hive.allow-drop-table=true +hive.allow-add-column=true +hive.allow-drop-column=true +hive.allow-rename-table=true +hive.allow-comment-table=true +hive.allow-rename-column=true +hive.metastore-cache-ttl=0s +hive.metastore.thrift.impersonation.enabled=true +hive.hdfs.authentication.type=NONE +hive.hdfs.impersonation.enabled=true +hive.fs.cache.max-size=10 +hive.max-partitions-per-scan=100 diff --git a/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-kerberos-hive-impersonation/hive.properties b/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-kerberos-hive-impersonation/hive.properties new file mode 100644 index 0000000000000..9c1d015fbe9f1 --- /dev/null +++ b/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-kerberos-hive-impersonation/hive.properties @@ -0,0 +1,28 @@ +# +# WARNING +# ^^^^^^^ +# This configuration file is for development only and should NOT be used +# in production. For example configuration, see the Presto documentation. +# + +connector.name=hive-hadoop2 +hive.metastore.uri=thrift://hadoop-master:9083 +hive.metastore.thrift.client.socks-proxy=hadoop-master:1180 +hive.config.resources=/docker/volumes/conf/presto/etc/hive-default-fs-site.xml +hive.metastore-cache-ttl=0s + +hive.metastore.authentication.type=KERBEROS +hive.metastore.thrift.impersonation.enabled=true +hive.metastore.service.principal=hive/hadoop-master@LABS.TERADATA.COM +hive.metastore.client.principal=hive/_HOST@LABS.TERADATA.COM +hive.metastore.client.keytab=/etc/presto/conf/hive-presto-master.keytab + +hive.hdfs.authentication.type=KERBEROS +hive.hdfs.impersonation.enabled=true +hive.hdfs.presto.principal=presto-server/_HOST@LABS.TERADATA.COM +hive.hdfs.presto.keytab=/etc/presto/conf/presto-server.keytab +hive.fs.cache.max-size=10 +hive.max-partitions-per-scan=100 + +#required for testGrantRevoke() product test +hive.security=sql-standard diff --git a/presto-product-tests/src/main/java/io/prestosql/tests/TestGroups.java b/presto-product-tests/src/main/java/io/prestosql/tests/TestGroups.java index 29b2c5caab296..2e03245c9b506 100644 --- a/presto-product-tests/src/main/java/io/prestosql/tests/TestGroups.java +++ b/presto-product-tests/src/main/java/io/prestosql/tests/TestGroups.java @@ -63,8 +63,6 @@ public final class TestGroups public static final String BIG_QUERY = "big_query"; public static final String HIVE_TABLE_STATISTICS = "hive_table_statistics"; public static final String KAFKA = "kafka"; - // TODO remove avro_schema_url group when adding Metastore impersonation - public static final String AVRO_SCHEMA_URL = "avro_schema_url"; private TestGroups() {} } diff --git a/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestAvroSchemaEvolution.java b/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestAvroSchemaEvolution.java index 4c88329b0ba9f..313eab396c121 100644 --- a/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestAvroSchemaEvolution.java +++ b/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestAvroSchemaEvolution.java @@ -23,7 +23,6 @@ import static io.prestosql.tempto.assertions.QueryAssert.assertThat; import static io.prestosql.tempto.context.ThreadLocalTestContextHolder.testContext; import static io.prestosql.tempto.query.QueryExecutor.query; -import static io.prestosql.tests.TestGroups.AVRO_SCHEMA_URL; import static java.lang.String.format; public class TestAvroSchemaEvolution @@ -60,14 +59,14 @@ public void dropTestTable() query(format("DROP TABLE IF EXISTS %s", TABLE_NAME)); } - @Test(groups = {AVRO_SCHEMA_URL}) + @Test public void testSelectTable() { assertThat(query(format("SELECT string_col FROM %s", TABLE_NAME))) .containsExactly(row("string0")); } - @Test(groups = {AVRO_SCHEMA_URL}) + @Test public void testInsertAfterSchemaEvolution() { assertThat(query(SELECT_STAR)) @@ -81,7 +80,7 @@ public void testInsertAfterSchemaEvolution() row("string1", 1, 101)); } - @Test(groups = {AVRO_SCHEMA_URL}) + @Test public void testSchemaEvolutionWithIncompatibleType() { assertThat(query(COLUMNS_IN_TABLE)) @@ -96,7 +95,7 @@ public void testSchemaEvolutionWithIncompatibleType() .failsWithMessage("Found int, expecting string"); } - @Test(groups = {AVRO_SCHEMA_URL}) + @Test public void testSchemaEvolution() { assertThat(query(COLUMNS_IN_TABLE)) @@ -138,7 +137,7 @@ public void testSchemaEvolution() .containsExactly(row("string0", null)); } - @Test(groups = {AVRO_SCHEMA_URL}) + @Test public void testSchemaWhenUrlIsUnset() { assertThat(query(COLUMNS_IN_TABLE)) @@ -154,7 +153,7 @@ public void testSchemaWhenUrlIsUnset() row("dummy_col", "varchar", "", "")); } - @Test(groups = {AVRO_SCHEMA_URL}) + @Test public void testCreateTableLike() { String createTableLikeName = "test_avro_like"; diff --git a/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestAvroSchemaUrl.java b/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestAvroSchemaUrl.java index fbe60e4745b2e..a454d4f983e89 100644 --- a/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestAvroSchemaUrl.java +++ b/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestAvroSchemaUrl.java @@ -31,7 +31,6 @@ import static com.google.common.base.Strings.isNullOrEmpty; import static io.prestosql.tempto.assertions.QueryAssert.Row.row; import static io.prestosql.tempto.assertions.QueryAssert.assertThat; -import static io.prestosql.tests.TestGroups.AVRO_SCHEMA_URL; import static io.prestosql.tests.TestGroups.STORAGE_FORMATS; import static io.prestosql.tests.utils.QueryExecutors.onHive; import static io.prestosql.tests.utils.QueryExecutors.onPresto; @@ -81,7 +80,7 @@ public Object[][] avroSchemaLocations() }; } - @Test(dataProvider = "avroSchemaLocations", groups = {AVRO_SCHEMA_URL, STORAGE_FORMATS}) + @Test(dataProvider = "avroSchemaLocations", groups = {STORAGE_FORMATS}) public void testHiveCreatedTable(String schemaLocation) { onHive().executeQuery("DROP TABLE IF EXISTS test_avro_schema_url_hive"); @@ -101,7 +100,7 @@ public void testHiveCreatedTable(String schemaLocation) onHive().executeQuery("DROP TABLE test_avro_schema_url_hive"); } - @Test(groups = {AVRO_SCHEMA_URL}) + @Test public void testAvroSchemaUrlInSerdeProperties() throws IOException { @@ -143,7 +142,7 @@ public void testAvroSchemaUrlInSerdeProperties() onHive().executeQuery("DROP TABLE test_avro_schema_url_in_serde_properties"); } - @Test(dataProvider = "avroSchemaLocations", groups = {AVRO_SCHEMA_URL, STORAGE_FORMATS}) + @Test(dataProvider = "avroSchemaLocations", groups = {STORAGE_FORMATS}) public void testPrestoCreatedTable(String schemaLocation) { onPresto().executeQuery("DROP TABLE IF EXISTS test_avro_schema_url_presto"); @@ -156,7 +155,7 @@ public void testPrestoCreatedTable(String schemaLocation) onPresto().executeQuery("DROP TABLE test_avro_schema_url_presto"); } - @Test(groups = {AVRO_SCHEMA_URL, STORAGE_FORMATS}) + @Test(groups = {STORAGE_FORMATS}) public void testTableWithLongColumnType() { onPresto().executeQuery("DROP TABLE IF EXISTS test_avro_schema_url_long_column"); @@ -185,7 +184,7 @@ public void testTableWithLongColumnType() onPresto().executeQuery("DROP TABLE test_avro_schema_url_long_column"); } - @Test(groups = {AVRO_SCHEMA_URL, STORAGE_FORMATS}) + @Test(groups = {STORAGE_FORMATS}) public void testPartitionedTableWithLongColumnType() { if (isOnHdp()) {