From c8c4daf7e07cbd3959e6c041c05d8a78fcb41e82 Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Fri, 11 Mar 2022 18:54:23 -0800 Subject: [PATCH 1/5] Add TestingThriftHiveMetastoreBuilder --- .../plugin/hive/TestHiveAlluxioMetastore.java | 4 +- .../metastore/thrift/ThriftHiveMetastore.java | 2 + .../trino/plugin/hive/AbstractTestHive.java | 50 ++------- .../hive/AbstractTestHiveFileSystem.java | 25 ++--- .../plugin/hive/BaseTestHiveOnDataLake.java | 31 +----- .../io/trino/plugin/hive/HiveTestUtils.java | 33 +++--- .../TestingThriftHiveMetastoreBuilder.java | 100 ++++++++++++++++++ .../cache/TestCachingHiveMetastore.java | 28 +---- .../thrift/TestingMetastoreLocator.java | 2 +- .../plugin/hive/s3/S3HiveQueryRunner.java | 43 +------- 10 files changed, 155 insertions(+), 163 deletions(-) create mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java diff --git a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveAlluxioMetastore.java b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveAlluxioMetastore.java index a428af384899..83aa89777121 100644 --- a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveAlluxioMetastore.java +++ b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveAlluxioMetastore.java @@ -15,7 +15,6 @@ import alluxio.client.table.TableMasterClient; import alluxio.conf.PropertyKey; -import io.trino.plugin.hive.authentication.NoHdfsAuthentication; import io.trino.plugin.hive.metastore.HiveMetastoreConfig; import io.trino.plugin.hive.metastore.alluxio.AlluxioHiveMetastore; import io.trino.plugin.hive.metastore.alluxio.AlluxioHiveMetastoreConfig; @@ -27,6 +26,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; public class TestHiveAlluxioMetastore extends AbstractTestHive @@ -60,7 +60,7 @@ public void setup(String host, String port, int hiveVersionMajor, String timeZon AlluxioHiveMetastoreConfig alluxioConfig = new AlluxioHiveMetastoreConfig(); alluxioConfig.setMasterAddress(this.alluxioAddress); TableMasterClient client = AlluxioMetastoreModule.createCatalogMasterClient(alluxioConfig); - hdfsEnvironment = new HdfsEnvironment(createTestHdfsConfiguration(), new HdfsConfig(), new NoHdfsAuthentication()); + hdfsEnvironment = HDFS_ENVIRONMENT; setup(SCHEMA, hiveConfig, new AlluxioHiveMetastore(client, new HiveMetastoreConfig()), hdfsEnvironment); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java index fa2ec5fd8398..43cc17366077 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.hive.metastore.thrift; +import com.google.common.annotations.VisibleForTesting; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.collect.ImmutableList; @@ -211,6 +212,7 @@ public ThriftHiveMetastore( authenticationConfig.getAuthenticationType() != ThriftMetastoreAuthenticationType.NONE); } + @VisibleForTesting public ThriftHiveMetastore( MetastoreLocator metastoreLocator, boolean hideDeltaLakeTables, diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java index ce6348249b87..1f47e4eb7c85 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java @@ -31,15 +31,10 @@ import io.trino.plugin.hive.LocationService.WriteInfo; import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.authentication.NoHdfsAuthentication; -import io.trino.plugin.hive.azure.HiveAzureConfig; -import io.trino.plugin.hive.azure.TrinoAzureConfigurationInitializer; import io.trino.plugin.hive.fs.DirectoryLister; -import io.trino.plugin.hive.gcs.GoogleGcsConfigurationInitializer; -import io.trino.plugin.hive.gcs.HiveGcsConfig; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveColumnStatistics; import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreConfig; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.HivePrincipal; import io.trino.plugin.hive.metastore.HivePrivilegeInfo; @@ -53,15 +48,9 @@ import io.trino.plugin.hive.metastore.Table; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.MetastoreLocator; -import io.trino.plugin.hive.metastore.thrift.TestingMetastoreLocator; -import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; import io.trino.plugin.hive.orc.OrcPageSource; import io.trino.plugin.hive.parquet.ParquetPageSource; import io.trino.plugin.hive.rcfile.RcFilePageSource; -import io.trino.plugin.hive.s3.HiveS3Config; -import io.trino.plugin.hive.s3.TrinoS3ConfigurationInitializer; import io.trino.plugin.hive.security.SqlStandardAccessControlMetadata; import io.trino.spi.Page; import io.trino.spi.TrinoException; @@ -238,6 +227,8 @@ import static io.trino.plugin.hive.HiveTableProperties.SORTED_BY_PROPERTY; import static io.trino.plugin.hive.HiveTableProperties.STORAGE_FORMAT_PROPERTY; import static io.trino.plugin.hive.HiveTableProperties.TRANSACTIONAL; +import static io.trino.plugin.hive.HiveTestUtils.HDFS_CONFIGURATION; +import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static io.trino.plugin.hive.HiveTestUtils.PAGE_SORTER; import static io.trino.plugin.hive.HiveTestUtils.SESSION; import static io.trino.plugin.hive.HiveTestUtils.arrayType; @@ -254,6 +245,7 @@ import static io.trino.plugin.hive.HiveType.HIVE_STRING; import static io.trino.plugin.hive.HiveType.toHiveType; import static io.trino.plugin.hive.LocationHandle.WriteMode.STAGE_AND_MOVE_TO_TARGET_DIRECTORY; +import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION; import static io.trino.plugin.hive.metastore.HiveColumnStatistics.createBinaryColumnStatistics; import static io.trino.plugin.hive.metastore.HiveColumnStatistics.createBooleanColumnStatistics; @@ -787,20 +779,13 @@ protected final void setup(String host, int port, String databaseName, String ti .setParquetTimeZone(timeZone) .setRcfileTimeZone(timeZone); - Optional proxy = Optional.ofNullable(System.getProperty("hive.metastore.thrift.client.socks-proxy")) - .map(HostAndPort::fromString); - - MetastoreLocator metastoreLocator = new TestingMetastoreLocator(proxy, HostAndPort.fromParts(host, port)); - - hdfsEnvironment = new HdfsEnvironment(createTestHdfsConfiguration(), new HdfsConfig(), new NoHdfsAuthentication()); + hdfsEnvironment = HDFS_ENVIRONMENT; HiveMetastore metastore = cachingHiveMetastore( - new BridgingHiveMetastore(new ThriftHiveMetastore( - metastoreLocator, - new HiveMetastoreConfig().isHideDeltaLakeTables(), - new HiveConfig().isTranslateHiveViews(), - new ThriftMetastoreConfig(), - hdfsEnvironment, - false), + new BridgingHiveMetastore(testingThriftHiveMetastoreBuilder() + .metastoreClient(HostAndPort.fromParts(host, port)) + .hiveConfig(hiveConfig) + .hdfsEnvironment(hdfsEnvironment) + .build(), new HiveIdentity(SESSION.getIdentity())), executor, new Duration(1, MINUTES), @@ -924,21 +909,6 @@ public Optional getMaterializedView(Connect TESTING_TYPE_MANAGER); } - protected HdfsConfiguration createTestHdfsConfiguration() - { - return new HiveHdfsConfiguration( - new HdfsConfigurationInitializer( - new HdfsConfig() - .setSocksProxy(Optional.ofNullable(System.getProperty("hive.hdfs.socks-proxy")) - .map(HostAndPort::fromString) - .orElse(null)), - ImmutableSet.of( - new TrinoS3ConfigurationInitializer(new HiveS3Config()), - new GoogleGcsConfigurationInitializer(new HiveGcsConfig()), - new TrinoAzureConfigurationInitializer(new HiveAzureConfig()))), - ImmutableSet.of()); - } - /** * Allow subclass to change default configuration. */ @@ -5754,7 +5724,7 @@ public void testNewDirectoryPermissions() HdfsConfig configWithSkip = new HdfsConfig(); configWithSkip.setNewDirectoryPermissions(HdfsConfig.SKIP_DIR_PERMISSIONS); HdfsEnvironment hdfsEnvironmentWithSkip = new HdfsEnvironment( - createTestHdfsConfiguration(), + HDFS_CONFIGURATION, configWithSkip, new NoHdfsAuthentication()); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java index ed428a6128a2..3d100c5fe5dd 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java @@ -41,10 +41,6 @@ import io.trino.plugin.hive.metastore.StorageFormat; import io.trino.plugin.hive.metastore.Table; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.MetastoreLocator; -import io.trino.plugin.hive.metastore.thrift.TestingMetastoreLocator; -import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; import io.trino.plugin.hive.security.SqlStandardAccessControlMetadata; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; @@ -108,6 +104,7 @@ import static io.trino.plugin.hive.HiveTestUtils.getHiveSessionProperties; import static io.trino.plugin.hive.HiveTestUtils.getTypes; import static io.trino.plugin.hive.HiveType.HIVE_LONG; +import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; import static io.trino.plugin.hive.metastore.PrincipalPrivileges.NO_PRIVILEGES; import static io.trino.plugin.hive.util.HiveWriteUtils.getRawFileSystem; import static io.trino.spi.connector.MetadataProvider.NOOP_METADATA_PROVIDER; @@ -183,24 +180,16 @@ protected void setup(String host, int port, String databaseName, boolean s3Selec config = new HiveConfig().setS3SelectPushdownEnabled(s3SelectPushdownEnabled); - Optional proxy = Optional.ofNullable(System.getProperty("hive.metastore.thrift.client.socks-proxy")) - .map(HostAndPort::fromString); - - MetastoreLocator metastoreLocator = new TestingMetastoreLocator(proxy, HostAndPort.fromParts(host, port)); - HivePartitionManager hivePartitionManager = new HivePartitionManager(config); hdfsEnvironment = new HdfsEnvironment(hdfsConfiguration, new HdfsConfig(), new NoHdfsAuthentication()); - HiveMetastoreConfig hiveMetastoreConfig = new HiveMetastoreConfig(); metastoreClient = new TestingHiveMetastore( new BridgingHiveMetastore( - new ThriftHiveMetastore( - metastoreLocator, - hiveMetastoreConfig.isHideDeltaLakeTables(), - new HiveConfig().isTranslateHiveViews(), - new ThriftMetastoreConfig(), - hdfsEnvironment, - false), + testingThriftHiveMetastoreBuilder() + .metastoreClient(HostAndPort.fromParts(host, port)) + .hiveConfig(config) + .hdfsEnvironment(hdfsEnvironment) + .build(), new HiveIdentity(getHiveSession(config).getIdentity())), getBasePath(), hdfsEnvironment); @@ -209,7 +198,7 @@ protected void setup(String host, int port, String databaseName, boolean s3Selec metadataFactory = new HiveMetadataFactory( new CatalogName("hive"), config, - hiveMetastoreConfig, + new HiveMetastoreConfig(), HiveMetastoreFactory.ofInstance(metastoreClient), hdfsEnvironment, hivePartitionManager, diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseTestHiveOnDataLake.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseTestHiveOnDataLake.java index 4d08fab1b6e1..6acd3072f1a7 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseTestHiveOnDataLake.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseTestHiveOnDataLake.java @@ -15,22 +15,15 @@ import com.amazonaws.services.s3.AmazonS3; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import io.airlift.units.DataSize; -import io.airlift.units.Duration; import io.trino.Session; import io.trino.plugin.hive.authentication.HiveIdentity; -import io.trino.plugin.hive.authentication.NoHdfsAuthentication; import io.trino.plugin.hive.containers.HiveMinioDataLake; import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreConfig; import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.PartitionWithStatistics; import io.trino.plugin.hive.metastore.Table; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.TestingMetastoreLocator; -import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; import io.trino.plugin.hive.s3.S3HiveQueryRunner; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; @@ -40,10 +33,9 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.Optional; -import java.util.concurrent.TimeUnit; import static io.airlift.units.DataSize.Unit.MEGABYTE; +import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; import static io.trino.testing.MaterializedResult.resultBuilder; import static io.trino.testing.sql.TestTable.randomTableSuffix; import static java.lang.String.format; @@ -76,21 +68,10 @@ protected QueryRunner createQueryRunner() this.dockerizedS3DataLake = closeAfterClass( new HiveMinioDataLake(bucketName, ImmutableMap.of(), hiveHadoopImage)); this.dockerizedS3DataLake.start(); - this.metastoreClient = new BridgingHiveMetastore(new ThriftHiveMetastore( - new TestingMetastoreLocator( - Optional.empty(), - this.dockerizedS3DataLake.getHiveHadoop().getHiveMetastoreEndpoint()), - new HiveMetastoreConfig().isHideDeltaLakeTables(), - new HiveConfig().isTranslateHiveViews(), - new ThriftMetastoreConfig(), - new HdfsEnvironment(new HiveHdfsConfiguration( - new HdfsConfigurationInitializer( - new HdfsConfig(), - ImmutableSet.of()), - ImmutableSet.of()), - new HdfsConfig(), - new NoHdfsAuthentication()), - false), + this.metastoreClient = new BridgingHiveMetastore( + testingThriftHiveMetastoreBuilder() + .metastoreClient(this.dockerizedS3DataLake.getHiveHadoop().getHiveMetastoreEndpoint()) + .build(), HiveIdentity.none()); return S3HiveQueryRunner.builder(dockerizedS3DataLake) .setHiveProperties( @@ -104,8 +85,6 @@ protected QueryRunner createQueryRunner() // This is required to reduce memory pressure to test writing large files .put("hive.s3.streaming.part-size", HIVE_S3_STREAMING_PART_SIZE.toString()) .buildOrThrow()) - // Increased timeout due to occasional slower responses in such setup - .setMetastoreTimeout(new Duration(20, TimeUnit.SECONDS)) .build(); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java index 7badec907bf3..b55e98c42bab 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import com.google.common.net.HostAndPort; import io.airlift.units.DataSize; import io.trino.operator.PagesIndex; import io.trino.operator.PagesIndexPageSorter; @@ -57,6 +58,7 @@ import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.List; +import java.util.Optional; import java.util.Set; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -72,7 +74,23 @@ private HiveTestUtils() {} public static final ConnectorSession SESSION = getHiveSession(new HiveConfig()); - public static final HdfsEnvironment HDFS_ENVIRONMENT = createTestHdfsEnvironment(); + public static final Optional SOCKS_PROXY = Optional.ofNullable(System.getProperty("hive.metastore.thrift.client.socks-proxy")) + .map(HostAndPort::fromString); + + public static final HiveHdfsConfiguration HDFS_CONFIGURATION = new HiveHdfsConfiguration( + new HdfsConfigurationInitializer( + new HdfsConfig() + .setSocksProxy(SOCKS_PROXY.orElse(null)), + ImmutableSet.of( + new TrinoS3ConfigurationInitializer(new HiveS3Config()), + new GoogleGcsConfigurationInitializer(new HiveGcsConfig()), + new TrinoAzureConfigurationInitializer(new HiveAzureConfig()))), + ImmutableSet.of()); + + public static final HdfsEnvironment HDFS_ENVIRONMENT = new HdfsEnvironment( + HDFS_CONFIGURATION, + new HdfsConfig(), + new NoHdfsAuthentication()); public static final PageSorter PAGE_SORTER = new PagesIndexPageSorter(new PagesIndex.TestingFactory(false)); @@ -174,19 +192,6 @@ public static HiveRecordCursorProvider createGenericHiveRecordCursorProvider(Hdf return new GenericHiveRecordCursorProvider(hdfsEnvironment, DataSize.of(100, MEGABYTE)); } - private static HdfsEnvironment createTestHdfsEnvironment() - { - HdfsConfiguration hdfsConfig = new HiveHdfsConfiguration( - new HdfsConfigurationInitializer( - new HdfsConfig(), - ImmutableSet.of( - new TrinoS3ConfigurationInitializer(new HiveS3Config()), - new GoogleGcsConfigurationInitializer(new HiveGcsConfig()), - new TrinoAzureConfigurationInitializer(new HiveAzureConfig()))), - ImmutableSet.of()); - return new HdfsEnvironment(hdfsConfig, new HdfsConfig(), new NoHdfsAuthentication()); - } - public static MapType mapType(Type keyType, Type valueType) { return (MapType) TESTING_TYPE_MANAGER.getParameterizedType(StandardTypes.MAP, ImmutableList.of( diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java new file mode 100644 index 000000000000..666d153b45e2 --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java @@ -0,0 +1,100 @@ +/* + * 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.trino.plugin.hive; + +import com.google.common.collect.ImmutableSet; +import com.google.common.net.HostAndPort; +import io.trino.plugin.hive.authentication.NoHdfsAuthentication; +import io.trino.plugin.hive.azure.HiveAzureConfig; +import io.trino.plugin.hive.azure.TrinoAzureConfigurationInitializer; +import io.trino.plugin.hive.gcs.GoogleGcsConfigurationInitializer; +import io.trino.plugin.hive.gcs.HiveGcsConfig; +import io.trino.plugin.hive.metastore.HiveMetastoreConfig; +import io.trino.plugin.hive.metastore.thrift.MetastoreLocator; +import io.trino.plugin.hive.metastore.thrift.TestingMetastoreLocator; +import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastore; +import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreClient; +import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; +import io.trino.plugin.hive.s3.HiveS3Config; +import io.trino.plugin.hive.s3.TrinoS3ConfigurationInitializer; + +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; + +public final class TestingThriftHiveMetastoreBuilder +{ + private static final HdfsEnvironment HDFS_ENVIRONMENT = new HdfsEnvironment( + new HiveHdfsConfiguration( + new HdfsConfigurationInitializer( + new HdfsConfig() + .setSocksProxy(HiveTestUtils.SOCKS_PROXY.orElse(null)), + ImmutableSet.of( + new TrinoS3ConfigurationInitializer(new HiveS3Config()), + new GoogleGcsConfigurationInitializer(new HiveGcsConfig()), + new TrinoAzureConfigurationInitializer(new HiveAzureConfig()))), + ImmutableSet.of()), + new HdfsConfig(), + new NoHdfsAuthentication()); + + private MetastoreLocator metastoreLocator; + private HiveConfig hiveConfig = new HiveConfig(); + private HdfsEnvironment hdfsEnvironment = HDFS_ENVIRONMENT; + + public static TestingThriftHiveMetastoreBuilder testingThriftHiveMetastoreBuilder() + { + return new TestingThriftHiveMetastoreBuilder(); + } + + private TestingThriftHiveMetastoreBuilder() {} + + public TestingThriftHiveMetastoreBuilder metastoreClient(HostAndPort address) + { + requireNonNull(address, "address is null"); + checkState(metastoreLocator == null, "Metastore client already set"); + metastoreLocator = new TestingMetastoreLocator(HiveTestUtils.SOCKS_PROXY, address); + return this; + } + + public TestingThriftHiveMetastoreBuilder metastoreClient(ThriftMetastoreClient client) + { + requireNonNull(client, "client is null"); + checkState(metastoreLocator == null, "Metastore client already set"); + metastoreLocator = token -> client; + return this; + } + + public TestingThriftHiveMetastoreBuilder hiveConfig(HiveConfig hiveConfig) + { + this.hiveConfig = requireNonNull(hiveConfig, "hiveConfig is null"); + return this; + } + + public TestingThriftHiveMetastoreBuilder hdfsEnvironment(HdfsEnvironment hdfsEnvironment) + { + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + return this; + } + + public ThriftHiveMetastore build() + { + checkState(metastoreLocator != null, "metastore client not set"); + return new ThriftHiveMetastore( + metastoreLocator, + new HiveMetastoreConfig().isHideDeltaLakeTables(), + hiveConfig.isTranslateHiveViews(), + new ThriftMetastoreConfig(), + hdfsEnvironment, + false); + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java index b122e959e130..0541386af390 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java @@ -20,23 +20,19 @@ import io.airlift.log.Logger; import io.airlift.units.Duration; import io.trino.plugin.hive.HiveColumnHandle; -import io.trino.plugin.hive.HiveConfig; import io.trino.plugin.hive.HiveMetastoreClosure; import io.trino.plugin.hive.PartitionStatistics; import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreConfig; import io.trino.plugin.hive.metastore.HivePrincipal; import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.Table; import io.trino.plugin.hive.metastore.UnimplementedHiveMetastore; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.MetastoreLocator; import io.trino.plugin.hive.metastore.thrift.MockThriftMetastoreClient; import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastore; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreClient; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreStats; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.Range; @@ -72,9 +68,9 @@ import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; import static io.trino.plugin.hive.HiveColumnHandle.createBaseColumn; import static io.trino.plugin.hive.HiveStorageFormat.TEXTFILE; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static io.trino.plugin.hive.HiveType.HIVE_STRING; import static io.trino.plugin.hive.HiveType.toHiveType; +import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; import static io.trino.plugin.hive.metastore.HiveColumnStatistics.createIntegerColumnStatistics; import static io.trino.plugin.hive.metastore.MetastoreUtil.computePartitionKeyFilter; import static io.trino.plugin.hive.metastore.StorageFormat.fromHiveStorageFormat; @@ -153,8 +149,9 @@ private ThriftHiveMetastore createThriftHiveMetastore() private static ThriftHiveMetastore createThriftHiveMetastore(ThriftMetastoreClient client) { - MetastoreLocator metastoreLocator = new MockMetastoreLocator(client); - return new ThriftHiveMetastore(metastoreLocator, new HiveMetastoreConfig().isHideDeltaLakeTables(), new HiveConfig().isTranslateHiveViews(), new ThriftMetastoreConfig(), HDFS_ENVIRONMENT, false); + return testingThriftHiveMetastoreBuilder() + .metastoreClient(client) + .build(); } @Test @@ -842,21 +839,4 @@ private CachingHiveMetastore createMetastoreWithDirectExecutor(CachingHiveMetast config.getMetastoreCacheMaximumSize(), config.isPartitionCacheEnabled()); } - - private static class MockMetastoreLocator - implements MetastoreLocator - { - private final ThriftMetastoreClient client; - - private MockMetastoreLocator(ThriftMetastoreClient client) - { - this.client = client; - } - - @Override - public ThriftMetastoreClient createMetastoreClient(Optional delegationToken) - { - return client; - } - } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestingMetastoreLocator.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestingMetastoreLocator.java index 2a7256d6e00f..3bd28e163e3d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestingMetastoreLocator.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestingMetastoreLocator.java @@ -27,7 +27,7 @@ public class TestingMetastoreLocator implements MetastoreLocator { private static final HiveMetastoreAuthentication AUTHENTICATION = new NoHiveMetastoreAuthentication(); - private static final Duration TIMEOUT = new Duration(10, SECONDS); + private static final Duration TIMEOUT = new Duration(20, SECONDS); private final DefaultThriftMetastoreClientFactory factory; private final HostAndPort address; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/S3HiveQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/S3HiveQueryRunner.java index 7e903d163c50..ec0b87325a4e 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/S3HiveQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/S3HiveQueryRunner.java @@ -13,31 +13,18 @@ */ package io.trino.plugin.hive.s3; -import com.google.common.collect.ImmutableSet; import com.google.common.net.HostAndPort; -import io.airlift.units.Duration; -import io.trino.plugin.hive.HdfsConfig; -import io.trino.plugin.hive.HdfsConfigurationInitializer; -import io.trino.plugin.hive.HdfsEnvironment; -import io.trino.plugin.hive.HiveConfig; -import io.trino.plugin.hive.HiveHdfsConfiguration; import io.trino.plugin.hive.HiveQueryRunner; import io.trino.plugin.hive.authentication.HiveIdentity; -import io.trino.plugin.hive.authentication.NoHdfsAuthentication; import io.trino.plugin.hive.containers.HiveMinioDataLake; -import io.trino.plugin.hive.metastore.HiveMetastoreConfig; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.TestingMetastoreLocator; -import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; import io.trino.testing.DistributedQueryRunner; import java.util.Locale; import java.util.Map; -import java.util.Optional; -import java.util.concurrent.TimeUnit; import static com.google.common.base.Preconditions.checkArgument; +import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; import static java.util.Objects.requireNonNull; public final class S3HiveQueryRunner @@ -92,7 +79,6 @@ public static class Builder extends HiveQueryRunner.Builder { private HostAndPort hiveMetastoreEndpoint; - private Duration metastoreTimeout = new Duration(10, TimeUnit.SECONDS); private String s3Endpoint; private String s3AccessKey; private String s3SecretKey; @@ -104,12 +90,6 @@ public Builder setHiveMetastoreEndpoint(HostAndPort hiveMetastoreEndpoint) return this; } - public Builder setMetastoreTimeout(Duration metastoreTimeout) - { - this.metastoreTimeout = metastoreTimeout; - return this; - } - public Builder setS3Endpoint(String s3Endpoint) { this.s3Endpoint = requireNonNull(s3Endpoint, "s3Endpoint is null"); @@ -151,23 +131,10 @@ public DistributedQueryRunner build() addHiveProperty("hive.s3.aws-secret-key", s3SecretKey); addHiveProperty("hive.s3.path-style-access", "true"); setMetastore(distributedQueryRunner -> new BridgingHiveMetastore( - new ThriftHiveMetastore( - new TestingMetastoreLocator( - Optional.empty(), - hiveMetastoreEndpoint, - metastoreTimeout), - new HiveMetastoreConfig().isHideDeltaLakeTables(), - new HiveConfig().isTranslateHiveViews(), - new ThriftMetastoreConfig(), - new HdfsEnvironment(new HiveHdfsConfiguration( - new HdfsConfigurationInitializer( - new HdfsConfig(), - ImmutableSet.of()), - ImmutableSet.of()), - new HdfsConfig(), - new NoHdfsAuthentication()), - false), - new HiveIdentity(distributedQueryRunner.getDefaultSession().getIdentity().toConnectorIdentity()))); + testingThriftHiveMetastoreBuilder() + .metastoreClient(hiveMetastoreEndpoint) + .build(), + new HiveIdentity(distributedQueryRunner.getDefaultSession().getIdentity().toConnectorIdentity()))); setInitialSchemasLocationBase("s3a://" + bucketName); // cannot use s3:// as Hive metastore is not configured to accept it return super.build(); } From f92ca3f87c28b70cd24756efaffb61d5a3470778 Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Fri, 11 Mar 2022 17:19:02 -0800 Subject: [PATCH 2/5] Move Hive metastore creation to TokenDelegationThriftMetastoreFactory This factory encapsulates all logic for thrift hive metastore client creation. --- ...DeltaLakePerTransactionMetastoreCache.java | 2 + .../metastore/thrift/ThriftHiveMetastore.java | 97 +------------- .../thrift/ThriftMetastoreModule.java | 1 + ...TokenDelegationThriftMetastoreFactory.java | 120 ++++++++++++++++++ .../TestingThriftHiveMetastoreBuilder.java | 11 +- 5 files changed, 135 insertions(+), 96 deletions(-) create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/TokenDelegationThriftMetastoreFactory.java diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java index 804a6a737570..abd4e1e78753 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java @@ -34,6 +34,7 @@ import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreAuthenticationModule; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreClientFactory; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; +import io.trino.plugin.hive.metastore.thrift.TokenDelegationThriftMetastoreFactory; import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.DistributedQueryRunner; import io.trino.tpch.TpchEntity; @@ -107,6 +108,7 @@ protected void setup(Binder binder) binder.bind(MetastoreLocator.class).to(StaticMetastoreLocator.class).in(Scopes.SINGLETON); configBinder(binder).bindConfig(StaticMetastoreConfig.class); configBinder(binder).bindConfig(ThriftMetastoreConfig.class); + binder.bind(TokenDelegationThriftMetastoreFactory.class); binder.bind(ThriftMetastore.class).to(ThriftHiveMetastore.class).in(Scopes.SINGLETON); newExporter(binder).export(ThriftMetastore.class).as((generator) -> generator.generatedNameOf(ThriftHiveMetastore.class)); install(new ThriftMetastoreAuthenticationModule()); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java index 43cc17366077..083c8310fac6 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java @@ -13,16 +13,11 @@ */ package io.trino.plugin.hive.metastore.thrift; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.util.concurrent.UncheckedExecutionException; import io.airlift.log.Logger; import io.airlift.units.Duration; -import io.trino.collect.cache.NonEvictableLoadingCache; import io.trino.plugin.hive.HdfsEnvironment; import io.trino.plugin.hive.HdfsEnvironment.HdfsContext; import io.trino.plugin.hive.HideDeltaLakeTables; @@ -43,7 +38,6 @@ import io.trino.plugin.hive.metastore.HivePrivilegeInfo; import io.trino.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege; import io.trino.plugin.hive.metastore.PartitionWithStatistics; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreAuthenticationConfig.ThriftMetastoreAuthenticationType; import io.trino.plugin.hive.util.RetryDriver; import io.trino.spi.TrinoException; import io.trino.spi.connector.SchemaNotFoundException; @@ -97,7 +91,6 @@ import javax.annotation.concurrent.ThreadSafe; import javax.inject.Inject; -import java.io.Closeable; import java.io.IOException; import java.util.Collections; import java.util.HashSet; @@ -117,7 +110,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.isNullOrEmpty; import static com.google.common.base.Throwables.propagateIfPossible; -import static com.google.common.base.Throwables.throwIfInstanceOf; import static com.google.common.base.Throwables.throwIfUnchecked; import static com.google.common.base.Verify.verify; import static com.google.common.base.Verify.verifyNotNull; @@ -126,7 +118,6 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.Iterables.getOnlyElement; import static com.google.common.collect.Sets.difference; -import static io.trino.collect.cache.SafeCaches.buildNonEvictableCache; import static io.trino.plugin.hive.HiveErrorCode.HIVE_METASTORE_ERROR; import static io.trino.plugin.hive.HiveErrorCode.HIVE_TABLE_LOCK_NOT_ACQUIRED; import static io.trino.plugin.hive.ViewReaderUtil.PRESTO_VIEW_FLAG; @@ -150,7 +141,6 @@ import static java.lang.String.format; import static java.lang.System.nanoTime; import static java.util.Objects.requireNonNull; -import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.hadoop.hive.common.FileUtils.makePartName; import static org.apache.hadoop.hive.metastore.TableType.MANAGED_TABLE; @@ -170,7 +160,7 @@ public class ThriftHiveMetastore private final ThriftMetastoreStats stats = new ThriftMetastoreStats(); private final HdfsEnvironment hdfsEnvironment; private final HdfsContext hdfsContext; - private final MetastoreLocator clientProvider; + private final TokenDelegationThriftMetastoreFactory metastoreFactory; private final double backoffScaleFactor; private final Duration minBackoffDelay; private final Duration maxBackoffDelay; @@ -178,8 +168,6 @@ public class ThriftHiveMetastore private final Duration maxWaitForLock; private final int maxRetries; private final boolean impersonationEnabled; - private final boolean authenticationEnabled; - private final NonEvictableLoadingCache delegationTokenCache; private final boolean deleteFilesOnDrop; private final boolean translateHiveViews; @@ -196,33 +184,14 @@ public class ThriftHiveMetastore @Inject public ThriftHiveMetastore( - MetastoreLocator metastoreLocator, + TokenDelegationThriftMetastoreFactory metastoreFactory, @HideDeltaLakeTables boolean hideDeltaLakeTables, @TranslateHiveViews boolean translateHiveViews, ThriftMetastoreConfig thriftConfig, - ThriftMetastoreAuthenticationConfig authenticationConfig, HdfsEnvironment hdfsEnvironment) - { - this( - metastoreLocator, - hideDeltaLakeTables, - translateHiveViews, - thriftConfig, - hdfsEnvironment, - authenticationConfig.getAuthenticationType() != ThriftMetastoreAuthenticationType.NONE); - } - - @VisibleForTesting - public ThriftHiveMetastore( - MetastoreLocator metastoreLocator, - boolean hideDeltaLakeTables, - boolean translateHiveViews, - ThriftMetastoreConfig thriftConfig, - HdfsEnvironment hdfsEnvironment, - boolean authenticationEnabled) { this.hdfsContext = new HdfsContext(ConnectorIdentity.ofUser(DEFAULT_METASTORE_USER)); - this.clientProvider = requireNonNull(metastoreLocator, "metastoreLocator is null"); + this.metastoreFactory = requireNonNull(metastoreFactory, "metastoreFactory is null"); this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.backoffScaleFactor = thriftConfig.getBackoffScaleFactor(); this.minBackoffDelay = thriftConfig.getMinBackoffDelay(); @@ -234,13 +203,7 @@ public ThriftHiveMetastore( this.translateHiveViews = translateHiveViews; checkArgument(!hideDeltaLakeTables, "Hiding Delta Lake tables is not supported"); // TODO this.maxWaitForLock = thriftConfig.getMaxWaitForTransactionLock(); - this.authenticationEnabled = authenticationEnabled; - this.delegationTokenCache = buildNonEvictableCache( - CacheBuilder.newBuilder() - .expireAfterWrite(thriftConfig.getDelegationTokenCacheTtl().toMillis(), MILLISECONDS) - .maximumSize(thriftConfig.getDelegationTokenCacheMaximumSize()), - CacheLoader.from(this::loadDelegationToken)); this.assumeCanonicalPartitionKeys = thriftConfig.isAssumeCanonicalPartitionKeys(); } @@ -2075,62 +2038,10 @@ private static boolean isUnknownMethodExceptionalResponse(Exception exception) return applicationException.getType() == UNKNOWN_METHOD; } - private ThriftMetastoreClient createMetastoreClient() - throws TException - { - return clientProvider.createMetastoreClient(Optional.empty()); - } - private ThriftMetastoreClient createMetastoreClient(HiveIdentity identity) throws TException { - if (!impersonationEnabled) { - return createMetastoreClient(); - } - - String username = identity.getUsername().orElseThrow(() -> new IllegalStateException("End-user name should exist when metastore impersonation is enabled")); - if (authenticationEnabled) { - String delegationToken; - try { - delegationToken = delegationTokenCache.getUnchecked(username); - } - catch (UncheckedExecutionException e) { - throwIfInstanceOf(e.getCause(), TrinoException.class); - throw e; - } - return clientProvider.createMetastoreClient(Optional.of(delegationToken)); - } - - ThriftMetastoreClient client = createMetastoreClient(); - setMetastoreUserOrClose(client, username); - return client; - } - - private String loadDelegationToken(String username) - { - try (ThriftMetastoreClient client = createMetastoreClient()) { - return client.getDelegationToken(username); - } - catch (TException e) { - throw new TrinoException(HIVE_METASTORE_ERROR, e); - } - } - - 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 - } - } + return metastoreFactory.createMetastoreClient(identity); } private RetryDriver retry() diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java index 54b039bb5b53..02b4545079bc 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java @@ -32,6 +32,7 @@ protected void setup(Binder binder) OptionalBinder.newOptionalBinder(binder, ThriftMetastoreClientFactory.class) .setDefault().to(DefaultThriftMetastoreClientFactory.class).in(Scopes.SINGLETON); binder.bind(MetastoreLocator.class).to(StaticMetastoreLocator.class).in(Scopes.SINGLETON); + binder.bind(TokenDelegationThriftMetastoreFactory.class); configBinder(binder).bindConfig(StaticMetastoreConfig.class); configBinder(binder).bindConfig(ThriftMetastoreConfig.class); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/TokenDelegationThriftMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/TokenDelegationThriftMetastoreFactory.java new file mode 100644 index 000000000000..e6adbe678eb4 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/TokenDelegationThriftMetastoreFactory.java @@ -0,0 +1,120 @@ +/* + * 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.trino.plugin.hive.metastore.thrift; + +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.util.concurrent.UncheckedExecutionException; +import io.trino.collect.cache.NonEvictableLoadingCache; +import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.authentication.HiveIdentity; +import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreAuthenticationConfig.ThriftMetastoreAuthenticationType; +import io.trino.spi.TrinoException; +import org.apache.thrift.TException; + +import javax.inject.Inject; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Optional; + +import static com.google.common.base.Throwables.throwIfInstanceOf; +import static io.trino.collect.cache.SafeCaches.buildNonEvictableCache; +import static io.trino.plugin.hive.HiveErrorCode.HIVE_METASTORE_ERROR; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +public class TokenDelegationThriftMetastoreFactory +{ + private final MetastoreLocator clientProvider; + private final boolean impersonationEnabled; + private final boolean authenticationEnabled; + private final NonEvictableLoadingCache delegationTokenCache; + + @Inject + public TokenDelegationThriftMetastoreFactory( + MetastoreLocator metastoreLocator, + ThriftMetastoreConfig thriftConfig, + ThriftMetastoreAuthenticationConfig authenticationConfig, + HdfsEnvironment hdfsEnvironment) + { + this.clientProvider = requireNonNull(metastoreLocator, "metastoreLocator is null"); + this.impersonationEnabled = thriftConfig.isImpersonationEnabled(); + this.authenticationEnabled = authenticationConfig.getAuthenticationType() != ThriftMetastoreAuthenticationType.NONE; + + this.delegationTokenCache = buildNonEvictableCache( + CacheBuilder.newBuilder() + .expireAfterWrite(thriftConfig.getDelegationTokenCacheTtl().toMillis(), MILLISECONDS) + .maximumSize(thriftConfig.getDelegationTokenCacheMaximumSize()), + CacheLoader.from(this::loadDelegationToken)); + } + + private ThriftMetastoreClient createMetastoreClient() + throws TException + { + return clientProvider.createMetastoreClient(Optional.empty()); + } + + public ThriftMetastoreClient createMetastoreClient(HiveIdentity identity) + throws TException + { + if (!impersonationEnabled) { + return createMetastoreClient(); + } + + String username = identity.getUsername().orElseThrow(() -> new IllegalStateException("End-user name should exist when metastore impersonation is enabled")); + if (authenticationEnabled) { + String delegationToken; + try { + delegationToken = delegationTokenCache.getUnchecked(username); + } + catch (UncheckedExecutionException e) { + throwIfInstanceOf(e.getCause(), TrinoException.class); + throw e; + } + return clientProvider.createMetastoreClient(Optional.of(delegationToken)); + } + + ThriftMetastoreClient client = createMetastoreClient(); + setMetastoreUserOrClose(client, username); + return client; + } + + private String loadDelegationToken(String username) + { + try (ThriftMetastoreClient client = createMetastoreClient()) { + return client.getDelegationToken(username); + } + catch (TException e) { + throw new TrinoException(HIVE_METASTORE_ERROR, e); + } + } + + 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 + } + } + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java index 666d153b45e2..98777f8330aa 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java @@ -24,8 +24,10 @@ import io.trino.plugin.hive.metastore.thrift.MetastoreLocator; import io.trino.plugin.hive.metastore.thrift.TestingMetastoreLocator; import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastore; +import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreAuthenticationConfig; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreClient; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; +import io.trino.plugin.hive.metastore.thrift.TokenDelegationThriftMetastoreFactory; import io.trino.plugin.hive.s3.HiveS3Config; import io.trino.plugin.hive.s3.TrinoS3ConfigurationInitializer; @@ -90,11 +92,14 @@ public ThriftHiveMetastore build() { checkState(metastoreLocator != null, "metastore client not set"); return new ThriftHiveMetastore( - metastoreLocator, + new TokenDelegationThriftMetastoreFactory( + metastoreLocator, + new ThriftMetastoreConfig(), + new ThriftMetastoreAuthenticationConfig(), + hdfsEnvironment), new HiveMetastoreConfig().isHideDeltaLakeTables(), hiveConfig.isTranslateHiveViews(), new ThriftMetastoreConfig(), - hdfsEnvironment, - false); + hdfsEnvironment); } } From 13ddb1f7d2f8068d305e6297f4153d4556d39df5 Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Sun, 13 Mar 2022 18:34:16 -0700 Subject: [PATCH 3/5] Cleanup ThriftHiveMetastore --- .../hive/metastore/thrift/ThriftHiveMetastore.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java index 083c8310fac6..8c5502240d50 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java @@ -157,9 +157,13 @@ public class ThriftHiveMetastore private static final int MAX_SET_DATE_STATISTICS_ATTEMPTS = 100; private static final String DEFAULT_METASTORE_USER = "presto"; + private static final Pattern TABLE_PARAMETER_SAFE_KEY_PATTERN = Pattern.compile("^[a-zA-Z_]+$"); + private static final Pattern TABLE_PARAMETER_SAFE_VALUE_PATTERN = Pattern.compile("^[a-zA-Z0-9\\s]*$"); + private final ThriftMetastoreStats stats = new ThriftMetastoreStats(); + private final HdfsContext hdfsContext = new HdfsContext(ConnectorIdentity.ofUser(DEFAULT_METASTORE_USER)); + private final HdfsEnvironment hdfsEnvironment; - private final HdfsContext hdfsContext; private final TokenDelegationThriftMetastoreFactory metastoreFactory; private final double backoffScaleFactor; private final Duration minBackoffDelay; @@ -170,6 +174,7 @@ public class ThriftHiveMetastore private final boolean impersonationEnabled; private final boolean deleteFilesOnDrop; private final boolean translateHiveViews; + private final boolean assumeCanonicalPartitionKeys; private final AtomicInteger chosenGetTableAlternative = new AtomicInteger(Integer.MAX_VALUE); private final AtomicInteger chosenTableParamAlternative = new AtomicInteger(Integer.MAX_VALUE); @@ -178,10 +183,6 @@ public class ThriftHiveMetastore private final AtomicReference> metastoreSupportsDateStatistics = new AtomicReference<>(Optional.empty()); private final CoalescingCounter metastoreSetDateStatisticsFailures = new CoalescingCounter(new Duration(1, SECONDS)); - private static final Pattern TABLE_PARAMETER_SAFE_KEY_PATTERN = Pattern.compile("^[a-zA-Z_]+$"); - private static final Pattern TABLE_PARAMETER_SAFE_VALUE_PATTERN = Pattern.compile("^[a-zA-Z0-9\\s]*$"); - private final boolean assumeCanonicalPartitionKeys; - @Inject public ThriftHiveMetastore( TokenDelegationThriftMetastoreFactory metastoreFactory, @@ -190,7 +191,6 @@ public ThriftHiveMetastore( ThriftMetastoreConfig thriftConfig, HdfsEnvironment hdfsEnvironment) { - this.hdfsContext = new HdfsContext(ConnectorIdentity.ofUser(DEFAULT_METASTORE_USER)); this.metastoreFactory = requireNonNull(metastoreFactory, "metastoreFactory is null"); this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.backoffScaleFactor = thriftConfig.getBackoffScaleFactor(); From 93fee3249c2fe51dd8e1ff21fafe8f6564fcffb2 Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Sat, 11 Jun 2022 13:52:44 -0700 Subject: [PATCH 4/5] Add ThriftMetastoreFactory --- ...DeltaLakePerTransactionMetastoreCache.java | 8 +- .../thrift/BridgingHiveMetastoreFactory.java | 10 +- .../metastore/thrift/ThriftHiveMetastore.java | 55 +++++------ .../thrift/ThriftHiveMetastoreFactory.java | 98 +++++++++++++++++++ .../metastore/thrift/ThriftMetastore.java | 2 - .../thrift/ThriftMetastoreFactory.java | 21 ++++ .../thrift/ThriftMetastoreModule.java | 4 +- .../TestingThriftHiveMetastoreBuilder.java | 8 +- .../cache/TestCachingHiveMetastore.java | 11 ++- .../thrift/InMemoryThriftMetastore.java | 6 -- .../HiveMetastoreTableOperationsProvider.java | 10 +- 11 files changed, 171 insertions(+), 62 deletions(-) create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastoreFactory.java create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreFactory.java diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java index abd4e1e78753..b45f52e0bfa5 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java @@ -30,10 +30,11 @@ import io.trino.plugin.hive.metastore.thrift.StaticMetastoreConfig; import io.trino.plugin.hive.metastore.thrift.StaticMetastoreLocator; import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastore; +import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastoreFactory; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreAuthenticationModule; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreClientFactory; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; +import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreFactory; import io.trino.plugin.hive.metastore.thrift.TokenDelegationThriftMetastoreFactory; import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.DistributedQueryRunner; @@ -109,8 +110,9 @@ protected void setup(Binder binder) configBinder(binder).bindConfig(StaticMetastoreConfig.class); configBinder(binder).bindConfig(ThriftMetastoreConfig.class); binder.bind(TokenDelegationThriftMetastoreFactory.class); - binder.bind(ThriftMetastore.class).to(ThriftHiveMetastore.class).in(Scopes.SINGLETON); - newExporter(binder).export(ThriftMetastore.class).as((generator) -> generator.generatedNameOf(ThriftHiveMetastore.class)); + binder.bind(ThriftMetastoreFactory.class).to(ThriftHiveMetastoreFactory.class).in(Scopes.SINGLETON); + newExporter(binder).export(ThriftMetastoreFactory.class) + .as(generator -> generator.generatedNameOf(ThriftHiveMetastore.class)); install(new ThriftMetastoreAuthenticationModule()); binder.bind(BridgingHiveMetastoreFactory.class).in(Scopes.SINGLETON); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java index f075d54205ea..5af6280951e4 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java @@ -27,23 +27,23 @@ public class BridgingHiveMetastoreFactory implements HiveMetastoreFactory { - private final ThriftMetastore thriftMetastore; + private final ThriftMetastoreFactory thriftMetastoreFactory; @Inject - public BridgingHiveMetastoreFactory(ThriftMetastore thriftMetastore) + public BridgingHiveMetastoreFactory(ThriftMetastoreFactory thriftMetastoreFactory) { - this.thriftMetastore = requireNonNull(thriftMetastore, "thriftMetastore is null"); + this.thriftMetastoreFactory = requireNonNull(thriftMetastoreFactory, "thriftMetastore is null"); } @Override public boolean isImpersonationEnabled() { - return thriftMetastore.isImpersonationEnabled(); + return thriftMetastoreFactory.isImpersonationEnabled(); } @Override public HiveMetastore createMetastore(Optional identity) { - return new BridgingHiveMetastore(thriftMetastore, identity.map(HiveIdentity::new).orElse(HiveIdentity.none())); + return new BridgingHiveMetastore(thriftMetastoreFactory.createMetastore(), identity.map(HiveIdentity::new).orElse(HiveIdentity.none())); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java index 8c5502240d50..fa48d229cf23 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.hive.metastore.thrift; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -20,7 +21,6 @@ import io.airlift.units.Duration; import io.trino.plugin.hive.HdfsEnvironment; import io.trino.plugin.hive.HdfsEnvironment.HdfsContext; -import io.trino.plugin.hive.HideDeltaLakeTables; import io.trino.plugin.hive.HiveBasicStatistics; import io.trino.plugin.hive.HivePartition; import io.trino.plugin.hive.HiveType; @@ -85,11 +85,8 @@ import org.apache.hadoop.hive.metastore.api.UnknownTableException; import org.apache.thrift.TApplicationException; import org.apache.thrift.TException; -import org.weakref.jmx.Flatten; -import org.weakref.jmx.Managed; import javax.annotation.concurrent.ThreadSafe; -import javax.inject.Inject; import java.io.IOException; import java.util.Collections; @@ -160,7 +157,6 @@ public class ThriftHiveMetastore private static final Pattern TABLE_PARAMETER_SAFE_KEY_PATTERN = Pattern.compile("^[a-zA-Z_]+$"); private static final Pattern TABLE_PARAMETER_SAFE_VALUE_PATTERN = Pattern.compile("^[a-zA-Z0-9\\s]*$"); - private final ThriftMetastoreStats stats = new ThriftMetastoreStats(); private final HdfsContext hdfsContext = new HdfsContext(ConnectorIdentity.ofUser(DEFAULT_METASTORE_USER)); private final HdfsEnvironment hdfsEnvironment; @@ -171,10 +167,10 @@ public class ThriftHiveMetastore private final Duration maxRetryTime; private final Duration maxWaitForLock; private final int maxRetries; - private final boolean impersonationEnabled; private final boolean deleteFilesOnDrop; private final boolean translateHiveViews; private final boolean assumeCanonicalPartitionKeys; + private final ThriftMetastoreStats stats; private final AtomicInteger chosenGetTableAlternative = new AtomicInteger(Integer.MAX_VALUE); private final AtomicInteger chosenTableParamAlternative = new AtomicInteger(Integer.MAX_VALUE); @@ -183,32 +179,35 @@ public class ThriftHiveMetastore private final AtomicReference> metastoreSupportsDateStatistics = new AtomicReference<>(Optional.empty()); private final CoalescingCounter metastoreSetDateStatisticsFailures = new CoalescingCounter(new Duration(1, SECONDS)); - @Inject public ThriftHiveMetastore( + HdfsEnvironment hdfsEnvironment, TokenDelegationThriftMetastoreFactory metastoreFactory, - @HideDeltaLakeTables boolean hideDeltaLakeTables, - @TranslateHiveViews boolean translateHiveViews, - ThriftMetastoreConfig thriftConfig, - HdfsEnvironment hdfsEnvironment) + double backoffScaleFactor, + Duration minBackoffDelay, + Duration maxBackoffDelay, + Duration maxRetryTime, + Duration maxWaitForLock, + int maxRetries, + boolean deleteFilesOnDrop, + boolean translateHiveViews, + boolean assumeCanonicalPartitionKeys, + ThriftMetastoreStats stats) { - this.metastoreFactory = requireNonNull(metastoreFactory, "metastoreFactory is null"); this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); - this.backoffScaleFactor = thriftConfig.getBackoffScaleFactor(); - this.minBackoffDelay = thriftConfig.getMinBackoffDelay(); - this.maxBackoffDelay = thriftConfig.getMaxBackoffDelay(); - this.maxRetryTime = thriftConfig.getMaxRetryTime(); - this.maxRetries = thriftConfig.getMaxRetries(); - this.impersonationEnabled = thriftConfig.isImpersonationEnabled(); - this.deleteFilesOnDrop = thriftConfig.isDeleteFilesOnDrop(); + this.metastoreFactory = requireNonNull(metastoreFactory, "metastoreFactory is null"); + this.backoffScaleFactor = backoffScaleFactor; + this.minBackoffDelay = requireNonNull(minBackoffDelay, "minBackoffDelay is null"); + this.maxBackoffDelay = requireNonNull(maxBackoffDelay, "maxBackoffDelay is null"); + this.maxRetryTime = requireNonNull(maxRetryTime, "maxRetryTime is null"); + this.maxWaitForLock = requireNonNull(maxWaitForLock, "maxWaitForLock is null"); + this.maxRetries = maxRetries; + this.deleteFilesOnDrop = deleteFilesOnDrop; this.translateHiveViews = translateHiveViews; - checkArgument(!hideDeltaLakeTables, "Hiding Delta Lake tables is not supported"); // TODO - this.maxWaitForLock = thriftConfig.getMaxWaitForTransactionLock(); - - this.assumeCanonicalPartitionKeys = thriftConfig.isAssumeCanonicalPartitionKeys(); + this.assumeCanonicalPartitionKeys = assumeCanonicalPartitionKeys; + this.stats = requireNonNull(stats, "stats is null"); } - @Managed - @Flatten + @VisibleForTesting public ThriftMetastoreStats getStats() { return stats; @@ -1935,12 +1934,6 @@ public void addDynamicPartitions(HiveIdentity identity, String dbName, String ta } } - @Override - public boolean isImpersonationEnabled() - { - return impersonationEnabled; - } - private static PrivilegeBag buildPrivilegeBag( String databaseName, String tableName, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastoreFactory.java new file mode 100644 index 000000000000..2d1cab174f10 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastoreFactory.java @@ -0,0 +1,98 @@ +/* + * 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.trino.plugin.hive.metastore.thrift; + +import io.airlift.units.Duration; +import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.HideDeltaLakeTables; +import org.weakref.jmx.Flatten; +import org.weakref.jmx.Managed; + +import javax.inject.Inject; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class ThriftHiveMetastoreFactory + implements ThriftMetastoreFactory +{ + private final HdfsEnvironment hdfsEnvironment; + private final TokenDelegationThriftMetastoreFactory metastoreFactory; + private final double backoffScaleFactor; + private final Duration minBackoffDelay; + private final Duration maxBackoffDelay; + private final Duration maxRetryTime; + private final Duration maxWaitForLock; + private final int maxRetries; + private final boolean impersonationEnabled; + private final boolean deleteFilesOnDrop; + private final boolean translateHiveViews; + private final boolean assumeCanonicalPartitionKeys; + private final ThriftMetastoreStats stats = new ThriftMetastoreStats(); + + @Inject + public ThriftHiveMetastoreFactory( + TokenDelegationThriftMetastoreFactory metastoreFactory, + @HideDeltaLakeTables boolean hideDeltaLakeTables, + @TranslateHiveViews boolean translateHiveViews, + ThriftMetastoreConfig thriftConfig, + HdfsEnvironment hdfsEnvironment) + { + this.metastoreFactory = requireNonNull(metastoreFactory, "metastoreFactory is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.backoffScaleFactor = thriftConfig.getBackoffScaleFactor(); + this.minBackoffDelay = thriftConfig.getMinBackoffDelay(); + this.maxBackoffDelay = thriftConfig.getMaxBackoffDelay(); + this.maxRetryTime = thriftConfig.getMaxRetryTime(); + this.maxRetries = thriftConfig.getMaxRetries(); + this.impersonationEnabled = thriftConfig.isImpersonationEnabled(); + this.deleteFilesOnDrop = thriftConfig.isDeleteFilesOnDrop(); + this.translateHiveViews = translateHiveViews; + checkArgument(!hideDeltaLakeTables, "Hiding Delta Lake tables is not supported"); // TODO + this.maxWaitForLock = thriftConfig.getMaxWaitForTransactionLock(); + + this.assumeCanonicalPartitionKeys = thriftConfig.isAssumeCanonicalPartitionKeys(); + } + + @Managed + @Flatten + public ThriftMetastoreStats getStats() + { + return stats; + } + + @Override + public boolean isImpersonationEnabled() + { + return impersonationEnabled; + } + + @Override + public ThriftMetastore createMetastore() + { + return new ThriftHiveMetastore( + hdfsEnvironment, + metastoreFactory, + backoffScaleFactor, + minBackoffDelay, + maxBackoffDelay, + maxRetryTime, + maxWaitForLock, + maxRetries, + deleteFilesOnDrop, + translateHiveViews, + assumeCanonicalPartitionKeys, + stats); + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastore.java index 1b251b211d31..3e978045fa16 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastore.java @@ -119,8 +119,6 @@ public interface ThriftMetastore */ Set listTablePrivileges(HiveIdentity identity, String databaseName, String tableName, Optional tableOwner, Optional principal); - boolean isImpersonationEnabled(); - default Optional> getFields(HiveIdentity identity, String databaseName, String tableName) { Optional table = getTable(identity, databaseName, tableName); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreFactory.java new file mode 100644 index 000000000000..4bae16f295bb --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreFactory.java @@ -0,0 +1,21 @@ +/* + * 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.trino.plugin.hive.metastore.thrift; + +public interface ThriftMetastoreFactory +{ + boolean isImpersonationEnabled(); + + ThriftMetastore createMetastore(); +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java index 02b4545079bc..d553f1a65600 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java @@ -36,8 +36,8 @@ protected void setup(Binder binder) configBinder(binder).bindConfig(StaticMetastoreConfig.class); configBinder(binder).bindConfig(ThriftMetastoreConfig.class); - binder.bind(ThriftMetastore.class).to(ThriftHiveMetastore.class).in(Scopes.SINGLETON); - newExporter(binder).export(ThriftMetastore.class) + binder.bind(ThriftMetastoreFactory.class).to(ThriftHiveMetastoreFactory.class).in(Scopes.SINGLETON); + newExporter(binder).export(ThriftMetastoreFactory.class) .as(generator -> generator.generatedNameOf(ThriftHiveMetastore.class)); binder.bind(HiveMetastoreFactory.class) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java index 98777f8330aa..3cfb0515f3a4 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java @@ -23,7 +23,8 @@ import io.trino.plugin.hive.metastore.HiveMetastoreConfig; import io.trino.plugin.hive.metastore.thrift.MetastoreLocator; import io.trino.plugin.hive.metastore.thrift.TestingMetastoreLocator; -import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastore; +import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastoreFactory; +import io.trino.plugin.hive.metastore.thrift.ThriftMetastore; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreAuthenticationConfig; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreClient; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; @@ -88,10 +89,10 @@ public TestingThriftHiveMetastoreBuilder hdfsEnvironment(HdfsEnvironment hdfsEnv return this; } - public ThriftHiveMetastore build() + public ThriftMetastore build() { checkState(metastoreLocator != null, "metastore client not set"); - return new ThriftHiveMetastore( + ThriftHiveMetastoreFactory metastoreFactory = new ThriftHiveMetastoreFactory( new TokenDelegationThriftMetastoreFactory( metastoreLocator, new ThriftMetastoreConfig(), @@ -101,5 +102,6 @@ public ThriftHiveMetastore build() hiveConfig.isTranslateHiveViews(), new ThriftMetastoreConfig(), hdfsEnvironment); + return metastoreFactory.createMetastore(); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java index 0541386af390..33cea081324f 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java @@ -32,6 +32,7 @@ import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; import io.trino.plugin.hive.metastore.thrift.MockThriftMetastoreClient; import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastore; +import io.trino.plugin.hive.metastore.thrift.ThriftMetastore; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreClient; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreStats; import io.trino.spi.predicate.Domain; @@ -122,7 +123,7 @@ public class TestCachingHiveMetastore public void setUp() { mockClient = new MockThriftMetastoreClient(); - ThriftHiveMetastore thriftHiveMetastore = createThriftHiveMetastore(); + ThriftMetastore thriftHiveMetastore = createThriftHiveMetastore(); executor = listeningDecorator(newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s"))); metastore = cachingHiveMetastore( new BridgingHiveMetastore(thriftHiveMetastore, IDENTITY), @@ -131,7 +132,7 @@ public void setUp() Optional.of(new Duration(1, TimeUnit.MINUTES)), 1000, true); - stats = thriftHiveMetastore.getStats(); + stats = ((ThriftHiveMetastore) thriftHiveMetastore).getStats(); } @AfterClass(alwaysRun = true) @@ -142,12 +143,12 @@ public void tearDown() metastore = null; } - private ThriftHiveMetastore createThriftHiveMetastore() + private ThriftMetastore createThriftHiveMetastore() { return createThriftHiveMetastore(mockClient); } - private static ThriftHiveMetastore createThriftHiveMetastore(ThriftMetastoreClient client) + private static ThriftMetastore createThriftHiveMetastore(ThriftMetastoreClient client) { return testingThriftHiveMetastoreBuilder() .metastoreClient(client) @@ -584,7 +585,7 @@ public void testCachingHiveMetastoreCreationWithTtlOnly() @Test public void testCachingHiveMetastoreCreationViaMemoize() { - ThriftHiveMetastore thriftHiveMetastore = createThriftHiveMetastore(); + ThriftMetastore thriftHiveMetastore = createThriftHiveMetastore(); metastore = memoizeMetastore( new BridgingHiveMetastore(thriftHiveMetastore, IDENTITY), 1000); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java index b441285e388d..e74003aae259 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java @@ -554,12 +554,6 @@ public void revokeTablePrivileges(HiveIdentity identity, String databaseName, St 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/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperationsProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperationsProvider.java index 76cb147056a2..1b56413d4cc9 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperationsProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperationsProvider.java @@ -14,7 +14,7 @@ package io.trino.plugin.iceberg.catalog.hms; import io.trino.plugin.hive.HdfsEnvironment.HdfsContext; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastore; +import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreFactory; import io.trino.plugin.iceberg.FileIoProvider; import io.trino.plugin.iceberg.catalog.IcebergTableOperations; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; @@ -31,13 +31,13 @@ public class HiveMetastoreTableOperationsProvider implements IcebergTableOperationsProvider { private final FileIoProvider fileIoProvider; - private final ThriftMetastore thriftMetastore; + private final ThriftMetastoreFactory thriftMetastoreFactory; @Inject - public HiveMetastoreTableOperationsProvider(FileIoProvider fileIoProvider, ThriftMetastore thriftMetastore) + public HiveMetastoreTableOperationsProvider(FileIoProvider fileIoProvider, ThriftMetastoreFactory thriftMetastoreFactory) { this.fileIoProvider = requireNonNull(fileIoProvider, "fileIoProvider is null"); - this.thriftMetastore = requireNonNull(thriftMetastore, "thriftMetastore is null"); + this.thriftMetastoreFactory = requireNonNull(thriftMetastoreFactory, "thriftMetastoreFactory is null"); } @Override @@ -52,7 +52,7 @@ public IcebergTableOperations createTableOperations( return new HiveMetastoreTableOperations( fileIoProvider.createFileIo(new HdfsContext(session), session.getQueryId()), ((TrinoHiveCatalog) catalog).getMetastore(), - thriftMetastore, + thriftMetastoreFactory.createMetastore(), session, database, table, From 4e24f761db2888101794d365936b9e24ed4e2fb5 Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Sat, 11 Jun 2022 15:07:28 -0700 Subject: [PATCH 5/5] Move ThriftHiveMetastore identity from method parameters to constructor Remove HiveIdentity --- .../hive/authentication/HiveIdentity.java | 79 ----- .../thrift/BridgingHiveMetastore.java | 120 ++++--- .../thrift/BridgingHiveMetastoreFactory.java | 3 +- .../metastore/thrift/ThriftHiveMetastore.java | 307 +++++++++--------- .../thrift/ThriftHiveMetastoreFactory.java | 6 +- .../metastore/thrift/ThriftMetastore.java | 96 +++--- .../thrift/ThriftMetastoreFactory.java | 6 +- ...TokenDelegationThriftMetastoreFactory.java | 7 +- .../trino/plugin/hive/AbstractTestHive.java | 4 +- .../hive/AbstractTestHiveFileSystem.java | 4 +- .../plugin/hive/AbstractTestHiveLocal.java | 7 +- .../plugin/hive/BaseTestHiveOnDataLake.java | 4 +- .../plugin/hive/TestHiveFileMetastore.java | 3 +- .../hive/TestHiveInMemoryMetastore.java | 5 +- .../TestingThriftHiveMetastoreBuilder.java | 4 +- .../cache/TestCachingHiveMetastore.java | 11 +- .../metastore/glue/TestHiveGlueMetastore.java | 3 +- .../thrift/InMemoryThriftMetastore.java | 81 +++-- .../plugin/hive/s3/S3HiveQueryRunner.java | 4 +- .../hms/HiveMetastoreTableOperations.java | 7 +- .../HiveMetastoreTableOperationsProvider.java | 2 +- 21 files changed, 327 insertions(+), 436 deletions(-) delete mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/authentication/HiveIdentity.java diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/authentication/HiveIdentity.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/authentication/HiveIdentity.java deleted file mode 100644 index 0c44779b674a..000000000000 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/authentication/HiveIdentity.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.trino.plugin.hive.authentication; - -import io.trino.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(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/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java index 224689581120..2edd98bcdc54 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java @@ -19,7 +19,6 @@ import io.trino.plugin.hive.PartitionStatistics; import io.trino.plugin.hive.acid.AcidOperation; import io.trino.plugin.hive.acid.AcidTransaction; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.AcidTransactionOwner; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; @@ -70,32 +69,30 @@ public class BridgingHiveMetastore implements HiveMetastore { private final ThriftMetastore delegate; - private final HiveIdentity identity; - public BridgingHiveMetastore(ThriftMetastore delegate, HiveIdentity identity) + public BridgingHiveMetastore(ThriftMetastore delegate) { this.delegate = requireNonNull(delegate, "delegate is null"); - this.identity = requireNonNull(identity, "identity is null"); } @Override public Optional getDatabase(String databaseName) { - return delegate.getDatabase(identity, databaseName).map(ThriftMetastoreUtil::fromMetastoreApiDatabase); + return delegate.getDatabase(databaseName).map(ThriftMetastoreUtil::fromMetastoreApiDatabase); } @Override public List getAllDatabases() { - return delegate.getAllDatabases(identity); + return delegate.getAllDatabases(); } @Override public Optional
getTable(String databaseName, String tableName) { - return delegate.getTable(identity, databaseName, tableName).map(table -> { + return delegate.getTable(databaseName, tableName).map(table -> { if (isAvroTableWithSchemaSet(table)) { - return fromMetastoreApiTable(table, delegate.getFields(identity, databaseName, tableName).orElseThrow()); + return fromMetastoreApiTable(table, delegate.getFields(databaseName, tableName).orElseThrow()); } if (isCsvTable(table)) { return fromMetastoreApiTable(table, csvSchemaFields(table.getSd().getCols())); @@ -113,14 +110,13 @@ public Set getSupportedColumnStatistics(Type type) @Override public PartitionStatistics getTableStatistics(Table table) { - return delegate.getTableStatistics(identity, toMetastoreApiTable(table)); + return delegate.getTableStatistics(toMetastoreApiTable(table)); } @Override public Map getPartitionStatistics(Table table, List partitions) { return delegate.getPartitionStatistics( - identity, toMetastoreApiTable(table), partitions.stream() .map(ThriftMetastoreUtil::toMetastoreApiPartition) @@ -130,55 +126,55 @@ public Map getPartitionStatistics(Table table, List @Override public void updateTableStatistics(String databaseName, String tableName, AcidTransaction transaction, Function update) { - delegate.updateTableStatistics(identity, databaseName, tableName, transaction, update); + delegate.updateTableStatistics(databaseName, tableName, transaction, update); } @Override public void updatePartitionStatistics(Table table, Map> updates) { org.apache.hadoop.hive.metastore.api.Table metastoreTable = toMetastoreApiTable(table); - updates.forEach((partitionName, update) -> delegate.updatePartitionStatistics(identity, metastoreTable, partitionName, update)); + updates.forEach((partitionName, update) -> delegate.updatePartitionStatistics(metastoreTable, partitionName, update)); } @Override public List getAllTables(String databaseName) { - return delegate.getAllTables(identity, databaseName); + return delegate.getAllTables(databaseName); } @Override public List getTablesWithParameter(String databaseName, String parameterKey, String parameterValue) { - return delegate.getTablesWithParameter(identity, databaseName, parameterKey, parameterValue); + return delegate.getTablesWithParameter(databaseName, parameterKey, parameterValue); } @Override public List getAllViews(String databaseName) { - return delegate.getAllViews(identity, databaseName); + return delegate.getAllViews(databaseName); } @Override public void createDatabase(Database database) { - delegate.createDatabase(identity, toMetastoreApiDatabase(database)); + delegate.createDatabase(toMetastoreApiDatabase(database)); } @Override public void dropDatabase(String databaseName, boolean deleteData) { - delegate.dropDatabase(identity, databaseName, deleteData); + delegate.dropDatabase(databaseName, deleteData); } @Override public void renameDatabase(String databaseName, String newDatabaseName) { - org.apache.hadoop.hive.metastore.api.Database database = delegate.getDatabase(identity, databaseName) + org.apache.hadoop.hive.metastore.api.Database database = delegate.getDatabase(databaseName) .orElseThrow(() -> new SchemaNotFoundException(databaseName)); database.setName(newDatabaseName); - delegate.alterDatabase(identity, databaseName, database); + delegate.alterDatabase(databaseName, database); - delegate.getDatabase(identity, databaseName).ifPresent(newDatabase -> { + delegate.getDatabase(databaseName).ifPresent(newDatabase -> { if (newDatabase.getName().equals(databaseName)) { throw new TrinoException(NOT_SUPPORTED, "Hive metastore does not support renaming schemas"); } @@ -188,7 +184,7 @@ public void renameDatabase(String databaseName, String newDatabaseName) @Override public void setDatabaseOwner(String databaseName, HivePrincipal principal) { - Database database = fromMetastoreApiDatabase(delegate.getDatabase(identity, databaseName) + Database database = fromMetastoreApiDatabase(delegate.getDatabase(databaseName) .orElseThrow(() -> new SchemaNotFoundException(databaseName))); Database newDatabase = Database.builder(database) @@ -196,19 +192,19 @@ public void setDatabaseOwner(String databaseName, HivePrincipal principal) .setOwnerType(Optional.of(principal.getType())) .build(); - delegate.alterDatabase(identity, databaseName, toMetastoreApiDatabase(newDatabase)); + delegate.alterDatabase(databaseName, toMetastoreApiDatabase(newDatabase)); } @Override public void createTable(Table table, PrincipalPrivileges principalPrivileges) { - delegate.createTable(identity, toMetastoreApiTable(table, principalPrivileges)); + delegate.createTable(toMetastoreApiTable(table, principalPrivileges)); } @Override public void dropTable(String databaseName, String tableName, boolean deleteData) { - delegate.dropTable(identity, databaseName, tableName, deleteData); + delegate.dropTable(databaseName, tableName, deleteData); } @Override @@ -220,7 +216,7 @@ public void replaceTable(String databaseName, String tableName, Table newTable, @Override public void renameTable(String databaseName, String tableName, String newDatabaseName, String newTableName) { - Optional source = delegate.getTable(identity, databaseName, tableName); + Optional source = delegate.getTable(databaseName, tableName); if (source.isEmpty()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -233,7 +229,7 @@ public void renameTable(String databaseName, String tableName, String newDatabas @Override public void commentTable(String databaseName, String tableName, Optional comment) { - Optional source = delegate.getTable(identity, databaseName, tableName); + Optional source = delegate.getTable(databaseName, tableName); if (source.isEmpty()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -256,20 +252,20 @@ public void setTableOwner(String databaseName, String tableName, HivePrincipal p throw new TrinoException(NOT_SUPPORTED, "Setting table owner type as a role is not supported"); } - Table table = fromMetastoreApiTable(delegate.getTable(identity, databaseName, tableName) + Table table = fromMetastoreApiTable(delegate.getTable(databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName)))); Table newTable = Table.builder(table) .setOwner(Optional.of(principal.getName())) .build(); - delegate.alterTable(identity, databaseName, tableName, toMetastoreApiTable(newTable)); + delegate.alterTable(databaseName, tableName, toMetastoreApiTable(newTable)); } @Override public void commentColumn(String databaseName, String tableName, String columnName, Optional comment) { - Optional source = delegate.getTable(identity, databaseName, tableName); + Optional source = delegate.getTable(databaseName, tableName); if (source.isEmpty()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -292,7 +288,7 @@ public void commentColumn(String databaseName, String tableName, String columnNa @Override public void addColumn(String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) { - Optional source = delegate.getTable(identity, databaseName, tableName); + Optional source = delegate.getTable(databaseName, tableName); if (source.isEmpty()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -305,7 +301,7 @@ public void addColumn(String databaseName, String tableName, String columnName, @Override public void renameColumn(String databaseName, String tableName, String oldColumnName, String newColumnName) { - Optional source = delegate.getTable(identity, databaseName, tableName); + Optional source = delegate.getTable(databaseName, tableName); if (source.isEmpty()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -327,7 +323,7 @@ public void renameColumn(String databaseName, String tableName, String oldColumn public void dropColumn(String databaseName, String tableName, String columnName) { verifyCanDropColumn(this, databaseName, tableName, columnName); - org.apache.hadoop.hive.metastore.api.Table table = delegate.getTable(identity, databaseName, tableName) + org.apache.hadoop.hive.metastore.api.Table table = delegate.getTable(databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); table.getSd().getCols().removeIf(fieldSchema -> fieldSchema.getName().equals(columnName)); alterTable(databaseName, tableName, table); @@ -335,13 +331,13 @@ public void dropColumn(String databaseName, String tableName, String columnName) private void alterTable(String databaseName, String tableName, org.apache.hadoop.hive.metastore.api.Table table) { - delegate.alterTable(identity, databaseName, tableName, table); + delegate.alterTable(databaseName, tableName, table); } @Override public Optional getPartition(Table table, List partitionValues) { - return delegate.getPartition(identity, table.getDatabaseName(), table.getTableName(), partitionValues).map(partition -> fromMetastoreApiPartition(table, partition)); + return delegate.getPartition(table.getDatabaseName(), table.getTableName(), partitionValues).map(partition -> fromMetastoreApiPartition(table, partition)); } @Override @@ -351,7 +347,7 @@ public Optional> getPartitionNamesByFilter( List columnNames, TupleDomain partitionKeysFilter) { - return delegate.getPartitionNamesByFilter(identity, databaseName, tableName, columnNames, partitionKeysFilter); + return delegate.getPartitionNamesByFilter(databaseName, tableName, columnNames, partitionKeysFilter); } @Override @@ -364,7 +360,7 @@ public Map> getPartitionsByNames(Table table, List> partitionNameToPartitionValuesMap = partitionNames.stream() .collect(Collectors.toMap(identity(), HiveUtil::toPartitionValues)); - Map, Partition> partitionValuesToPartitionMap = delegate.getPartitionsByNames(identity, table.getDatabaseName(), table.getTableName(), partitionNames).stream() + Map, Partition> partitionValuesToPartitionMap = delegate.getPartitionsByNames(table.getDatabaseName(), table.getTableName(), partitionNames).stream() .map(partition -> fromMetastoreApiPartition(table, partition)) .collect(Collectors.toMap(Partition::getValues, identity())); ImmutableMap.Builder> resultBuilder = ImmutableMap.builder(); @@ -390,109 +386,109 @@ private Partition fromMetastoreApiPartition(Table table, org.apache.hadoop.hive. @Override public void addPartitions(String databaseName, String tableName, List partitions) { - delegate.addPartitions(identity, databaseName, tableName, partitions); + delegate.addPartitions(databaseName, tableName, partitions); } @Override public void dropPartition(String databaseName, String tableName, List parts, boolean deleteData) { - delegate.dropPartition(identity, databaseName, tableName, parts, deleteData); + delegate.dropPartition(databaseName, tableName, parts, deleteData); } @Override public void alterPartition(String databaseName, String tableName, PartitionWithStatistics partition) { - delegate.alterPartition(identity, databaseName, tableName, partition); + delegate.alterPartition(databaseName, tableName, partition); } @Override public void createRole(String role, String grantor) { - delegate.createRole(identity, role, grantor); + delegate.createRole(role, grantor); } @Override public void dropRole(String role) { - delegate.dropRole(identity, role); + delegate.dropRole(role); } @Override public Set listRoles() { - return delegate.listRoles(identity); + return delegate.listRoles(); } @Override public void grantRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) { - delegate.grantRoles(identity, roles, grantees, adminOption, grantor); + delegate.grantRoles(roles, grantees, adminOption, grantor); } @Override public void revokeRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) { - delegate.revokeRoles(identity, roles, grantees, adminOption, grantor); + delegate.revokeRoles(roles, grantees, adminOption, grantor); } @Override public Set listGrantedPrincipals(String role) { - return delegate.listGrantedPrincipals(identity, role); + return delegate.listGrantedPrincipals(role); } @Override public Set listRoleGrants(HivePrincipal principal) { - return delegate.listRoleGrants(identity, principal); + return delegate.listRoleGrants(principal); } @Override public void grantTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) { - delegate.grantTablePrivileges(identity, databaseName, tableName, tableOwner, grantee, grantor, privileges, grantOption); + delegate.grantTablePrivileges(databaseName, tableName, tableOwner, grantee, grantor, privileges, grantOption); } @Override public void revokeTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) { - delegate.revokeTablePrivileges(identity, databaseName, tableName, tableOwner, grantee, grantor, privileges, grantOption); + delegate.revokeTablePrivileges(databaseName, tableName, tableOwner, grantee, grantor, privileges, grantOption); } @Override public Set listTablePrivileges(String databaseName, String tableName, Optional tableOwner, Optional principal) { - return delegate.listTablePrivileges(identity, databaseName, tableName, tableOwner, principal); + return delegate.listTablePrivileges(databaseName, tableName, tableOwner, principal); } @Override public Optional getConfigValue(String name) { - return delegate.getConfigValue(identity, name); + return delegate.getConfigValue(name); } @Override public long openTransaction(AcidTransactionOwner transactionOwner) { - return delegate.openTransaction(identity, transactionOwner); + return delegate.openTransaction(transactionOwner); } @Override public void commitTransaction(long transactionId) { - delegate.commitTransaction(identity, transactionId); + delegate.commitTransaction(transactionId); } @Override public void abortTransaction(long transactionId) { - delegate.abortTransaction(identity, transactionId); + delegate.abortTransaction(transactionId); } @Override public void sendTransactionHeartbeat(long transactionId) { - delegate.sendTransactionHeartbeat(identity, transactionId); + delegate.sendTransactionHeartbeat(transactionId); } @Override @@ -503,19 +499,19 @@ public void acquireSharedReadLock( List fullTables, List partitions) { - delegate.acquireSharedReadLock(identity, transactionOwner, queryId, transactionId, fullTables, partitions); + delegate.acquireSharedReadLock(transactionOwner, queryId, transactionId, fullTables, partitions); } @Override public String getValidWriteIds(List tables, long currentTransactionId) { - return delegate.getValidWriteIds(identity, tables, currentTransactionId); + return delegate.getValidWriteIds(tables, currentTransactionId); } @Override public long allocateWriteId(String dbName, String tableName, long transactionId) { - return delegate.allocateWriteId(identity, dbName, tableName, transactionId); + return delegate.allocateWriteId(dbName, tableName, transactionId); } @Override @@ -528,13 +524,13 @@ public void acquireTableWriteLock( DataOperationType operation, boolean isDynamicPartitionWrite) { - delegate.acquireTableWriteLock(identity, transactionOwner, queryId, transactionId, dbName, tableName, operation, isDynamicPartitionWrite); + delegate.acquireTableWriteLock(transactionOwner, queryId, transactionId, dbName, tableName, operation, isDynamicPartitionWrite); } @Override public void updateTableWriteId(String dbName, String tableName, long transactionId, long writeId, OptionalLong rowCountChange) { - delegate.updateTableWriteId(identity, dbName, tableName, transactionId, writeId, rowCountChange); + delegate.updateTableWriteId(dbName, tableName, transactionId, writeId, rowCountChange); } @Override @@ -544,18 +540,18 @@ public void alterPartitions(String dbName, String tableName, List par .map(ThriftMetastoreUtil::toMetastoreApiPartition) .peek(partition -> partition.setWriteId(writeId)) .collect(toImmutableList()); - delegate.alterPartitions(identity, dbName, tableName, hadoopPartitions, writeId); + delegate.alterPartitions(dbName, tableName, hadoopPartitions, writeId); } @Override public void addDynamicPartitions(String dbName, String tableName, List partitionNames, long transactionId, long writeId, AcidOperation operation) { - delegate.addDynamicPartitions(identity, dbName, tableName, partitionNames, transactionId, writeId, operation); + delegate.addDynamicPartitions(dbName, tableName, partitionNames, transactionId, writeId, operation); } @Override public void alterTransactionalTable(Table table, long transactionId, long writeId, PrincipalPrivileges principalPrivileges) { - delegate.alterTransactionalTable(identity, toMetastoreApiTable(table, principalPrivileges), transactionId, writeId); + delegate.alterTransactionalTable(toMetastoreApiTable(table, principalPrivileges), transactionId, writeId); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java index 5af6280951e4..404ef6c729f5 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.hive.metastore.thrift; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.security.ConnectorIdentity; @@ -44,6 +43,6 @@ public boolean isImpersonationEnabled() @Override public HiveMetastore createMetastore(Optional identity) { - return new BridgingHiveMetastore(thriftMetastoreFactory.createMetastore(), identity.map(HiveIdentity::new).orElse(HiveIdentity.none())); + return new BridgingHiveMetastore(thriftMetastoreFactory.createMetastore(identity)); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java index fa48d229cf23..b31b38e554f8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java @@ -30,7 +30,6 @@ import io.trino.plugin.hive.TableAlreadyExistsException; import io.trino.plugin.hive.acid.AcidOperation; import io.trino.plugin.hive.acid.AcidTransaction; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.AcidTransactionOwner; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveColumnStatistics; @@ -159,6 +158,7 @@ public class ThriftHiveMetastore private final HdfsContext hdfsContext = new HdfsContext(ConnectorIdentity.ofUser(DEFAULT_METASTORE_USER)); + private final Optional identity; private final HdfsEnvironment hdfsEnvironment; private final TokenDelegationThriftMetastoreFactory metastoreFactory; private final double backoffScaleFactor; @@ -180,6 +180,7 @@ public class ThriftHiveMetastore private final CoalescingCounter metastoreSetDateStatisticsFailures = new CoalescingCounter(new Duration(1, SECONDS)); public ThriftHiveMetastore( + Optional identity, HdfsEnvironment hdfsEnvironment, TokenDelegationThriftMetastoreFactory metastoreFactory, double backoffScaleFactor, @@ -193,6 +194,7 @@ public ThriftHiveMetastore( boolean assumeCanonicalPartitionKeys, ThriftMetastoreStats stats) { + this.identity = requireNonNull(identity, "identity is null"); this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.metastoreFactory = requireNonNull(metastoreFactory, "metastoreFactory is null"); this.backoffScaleFactor = backoffScaleFactor; @@ -214,13 +216,13 @@ public ThriftMetastoreStats getStats() } @Override - public List getAllDatabases(HiveIdentity identity) + public List getAllDatabases() { try { return retry() .stopOnIllegalExceptions() .run("getAllDatabases", stats.getGetAllDatabases().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return client.getAllDatabases(); } })); @@ -234,14 +236,14 @@ public List getAllDatabases(HiveIdentity identity) } @Override - public Optional getDatabase(HiveIdentity identity, String databaseName) + public Optional getDatabase(String databaseName) { try { return retry() .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getDatabase", stats.getGetDatabase().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return Optional.of(client.getDatabase(databaseName)); } })); @@ -258,14 +260,14 @@ public Optional getDatabase(HiveIdentity identity, String databaseName } @Override - public List getAllTables(HiveIdentity identity, String databaseName) + public List getAllTables(String databaseName) { try { return retry() .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getAllTables", () -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return client.getAllTables(databaseName); } }); @@ -282,14 +284,14 @@ public List getAllTables(HiveIdentity identity, String databaseName) } @Override - public List getTablesWithParameter(HiveIdentity identity, String databaseName, String parameterKey, String parameterValue) + public List getTablesWithParameter(String databaseName, String parameterKey, String parameterValue) { try { return retry() .stopOn(UnknownDBException.class) .stopOnIllegalExceptions() .run("getTablesWithParameter", stats.getGetTablesWithParameter().wrap( - () -> doGetTablesWithParameter(identity, databaseName, parameterKey, parameterValue))); + () -> doGetTablesWithParameter(databaseName, parameterKey, parameterValue))); } catch (UnknownDBException e) { return ImmutableList.of(); @@ -303,14 +305,14 @@ public List getTablesWithParameter(HiveIdentity identity, String databas } @Override - public Optional
getTable(HiveIdentity identity, String databaseName, String tableName) + public Optional
getTable(String databaseName, String tableName) { try { return retry() .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getTable", stats.getGetTable().wrap(() -> { - Table table = getTableFromMetastore(identity, databaseName, tableName); + Table table = getTableFromMetastore(databaseName, tableName); return Optional.of(table); })); } @@ -325,11 +327,11 @@ public Optional
getTable(HiveIdentity identity, String databaseName, Stri } } - private Table getTableFromMetastore(HiveIdentity identity, String databaseName, String tableName) + private Table getTableFromMetastore(String databaseName, String tableName) throws TException { return alternativeCall( - () -> createMetastoreClient(identity), + this::createMetastoreClient, ThriftHiveMetastore::defaultIsValidExceptionalResponse, chosenGetTableAlternative, client -> client.getTableWithCapabilities(databaseName, tableName), @@ -343,24 +345,24 @@ public Set getSupportedColumnStatistics(Type type) } @Override - public PartitionStatistics getTableStatistics(HiveIdentity identity, Table table) + public PartitionStatistics getTableStatistics(Table table) { List dataColumns = table.getSd().getCols().stream() .map(FieldSchema::getName) .collect(toImmutableList()); HiveBasicStatistics basicStatistics = getHiveBasicStatistics(table.getParameters()); - Map columnStatistics = getTableColumnStatistics(identity, table.getDbName(), table.getTableName(), dataColumns, basicStatistics.getRowCount()); + Map columnStatistics = getTableColumnStatistics(table.getDbName(), table.getTableName(), dataColumns, basicStatistics.getRowCount()); return new PartitionStatistics(basicStatistics, columnStatistics); } - private Map getTableColumnStatistics(HiveIdentity identity, String databaseName, String tableName, List columns, OptionalLong rowCount) + private Map getTableColumnStatistics(String databaseName, String tableName, List columns, OptionalLong rowCount) { try { return retry() .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getTableColumnStatistics", stats.getGetTableColumnStatistics().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return groupStatisticsByColumn(client.getTableColumnStatistics(databaseName, tableName, columns), rowCount); } })); @@ -377,7 +379,7 @@ private Map getTableColumnStatistics(HiveIdentity } @Override - public Map getPartitionStatistics(HiveIdentity identity, Table table, List partitions) + public Map getPartitionStatistics(Table table, List partitions) { List dataColumns = table.getSd().getCols().stream() .map(FieldSchema::getName) @@ -393,7 +395,6 @@ public Map getPartitionStatistics(HiveIdentity iden Map partitionRowCounts = partitionBasicStatistics.entrySet().stream() .collect(toImmutableMap(Map.Entry::getKey, entry -> entry.getValue().getRowCount())); Map> partitionColumnStatistics = getPartitionColumnStatistics( - identity, table.getDbName(), table.getTableName(), partitionBasicStatistics.keySet(), @@ -410,14 +411,14 @@ public Map getPartitionStatistics(HiveIdentity iden } @Override - public Optional> getFields(HiveIdentity identity, String databaseName, String tableName) + public Optional> getFields(String databaseName, String tableName) { try { return retry() .stopOn(MetaException.class, UnknownTableException.class, UnknownDBException.class) .stopOnIllegalExceptions() .run("getFields", stats.getGetFields().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return Optional.of(ImmutableList.copyOf(client.getFields(databaseName, tableName))); } })); @@ -434,28 +435,27 @@ public Optional> getFields(HiveIdentity identity, String datab } private Map> getPartitionColumnStatistics( - HiveIdentity identity, String databaseName, String tableName, Set partitionNames, List columnNames, Map partitionRowCounts) { - return getMetastorePartitionColumnStatistics(identity, databaseName, tableName, partitionNames, columnNames).entrySet().stream() + return getMetastorePartitionColumnStatistics(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(HiveIdentity identity, String databaseName, String tableName, Set partitionNames, List columnNames) + private Map> getMetastorePartitionColumnStatistics(String databaseName, String tableName, Set partitionNames, List columnNames) { try { return retry() .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getPartitionColumnStatistics", stats.getGetPartitionColumnStatistics().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return client.getPartitionColumnStatistics(databaseName, tableName, ImmutableList.copyOf(partitionNames), columnNames); } })); @@ -478,12 +478,12 @@ private static Map groupStatisticsByColumn(List update) + public void updateTableStatistics(String databaseName, String tableName, AcidTransaction transaction, Function update) { - Table originalTable = getTable(identity, databaseName, tableName) + Table originalTable = getTable(databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); - PartitionStatistics currentStatistics = getTableStatistics(identity, originalTable); + PartitionStatistics currentStatistics = getTableStatistics(originalTable); PartitionStatistics updatedStatistics = update.apply(currentStatistics); Table modifiedTable = originalTable.deepCopy(); @@ -492,7 +492,7 @@ public void updateTableStatistics(HiveIdentity identity, String databaseName, St if (transaction.isAcidTransactionRunning()) { modifiedTable.setWriteId(transaction.getWriteId()); } - alterTable(identity, databaseName, tableName, modifiedTable); + alterTable(databaseName, tableName, modifiedTable); io.trino.plugin.hive.metastore.Table table = fromMetastoreApiTable(modifiedTable); OptionalLong rowCount = basicStatistics.getRowCount(); @@ -510,13 +510,13 @@ public void updateTableStatistics(HiveIdentity identity, String databaseName, St }) .collect(toImmutableList()); if (!metastoreColumnStatistics.isEmpty()) { - setTableColumnStatistics(identity, databaseName, tableName, metastoreColumnStatistics); + setTableColumnStatistics(databaseName, tableName, metastoreColumnStatistics); } Set removedColumnStatistics = difference(currentStatistics.getColumnStatistics().keySet(), updatedStatistics.getColumnStatistics().keySet()); - removedColumnStatistics.forEach(column -> deleteTableColumnStatistics(identity, databaseName, tableName, column)); + removedColumnStatistics.forEach(column -> deleteTableColumnStatistics(databaseName, tableName, column)); } - private void setTableColumnStatistics(HiveIdentity identity, String databaseName, String tableName, List statistics) + private void setTableColumnStatistics(String databaseName, String tableName, List statistics) { try { retry() @@ -524,7 +524,6 @@ private void setTableColumnStatistics(HiveIdentity identity, String databaseName .stopOnIllegalExceptions() .run("setTableColumnStatistics", stats.getSetTableColumnStatistics().wrap(() -> { setColumnStatistics( - identity, format("table %s.%s", databaseName, tableName), statistics, (client, stats) -> client.setTableColumnStatistics(databaseName, tableName, stats)); @@ -542,14 +541,14 @@ private void setTableColumnStatistics(HiveIdentity identity, String databaseName } } - private void deleteTableColumnStatistics(HiveIdentity identity, String databaseName, String tableName, String columnName) + private void deleteTableColumnStatistics(String databaseName, String tableName, String columnName) { try { retry() .stopOn(NoSuchObjectException.class, InvalidObjectException.class, MetaException.class, InvalidInputException.class) .stopOnIllegalExceptions() .run("deleteTableColumnStatistics", stats.getDeleteTableColumnStatistics().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.deleteTableColumnStatistics(databaseName, tableName, columnName); } return null; @@ -567,33 +566,32 @@ private void deleteTableColumnStatistics(HiveIdentity identity, String databaseN } @Override - public void updatePartitionStatistics(HiveIdentity identity, Table table, String partitionName, Function update) + public void updatePartitionStatistics(Table table, String partitionName, Function update) { - List partitions = getPartitionsByNames(identity, table.getDbName(), table.getTableName(), ImmutableList.of(partitionName)); + List partitions = getPartitionsByNames(table.getDbName(), table.getTableName(), ImmutableList.of(partitionName)); if (partitions.size() != 1) { throw new TrinoException(HIVE_METASTORE_ERROR, "Metastore returned multiple partitions for name: " + partitionName); } Partition originalPartition = getOnlyElement(partitions); PartitionStatistics currentStatistics = requireNonNull( - getPartitionStatistics(identity, table, partitions).get(partitionName), "getPartitionStatistics() did not return statistics for partition"); + getPartitionStatistics(table, partitions).get(partitionName), "getPartitionStatistics() did not return statistics for partition"); PartitionStatistics updatedStatistics = update.apply(currentStatistics); Partition modifiedPartition = originalPartition.deepCopy(); HiveBasicStatistics basicStatistics = updatedStatistics.getBasicStatistics(); modifiedPartition.setParameters(updateStatisticsParameters(modifiedPartition.getParameters(), basicStatistics)); - alterPartitionWithoutStatistics(identity, table.getDbName(), table.getTableName(), modifiedPartition); + alterPartitionWithoutStatistics(table.getDbName(), table.getTableName(), modifiedPartition); Map columns = modifiedPartition.getSd().getCols().stream() .collect(toImmutableMap(FieldSchema::getName, schema -> HiveType.valueOf(schema.getType()))); - setPartitionColumnStatistics(identity, table.getDbName(), table.getTableName(), partitionName, columns, updatedStatistics.getColumnStatistics(), basicStatistics.getRowCount()); + setPartitionColumnStatistics(table.getDbName(), table.getTableName(), partitionName, columns, updatedStatistics.getColumnStatistics(), basicStatistics.getRowCount()); Set removedStatistics = difference(currentStatistics.getColumnStatistics().keySet(), updatedStatistics.getColumnStatistics().keySet()); - removedStatistics.forEach(column -> deletePartitionColumnStatistics(identity, table.getDbName(), table.getTableName(), partitionName, column)); + removedStatistics.forEach(column -> deletePartitionColumnStatistics(table.getDbName(), table.getTableName(), partitionName, column)); } private void setPartitionColumnStatistics( - HiveIdentity identity, String databaseName, String tableName, String partitionName, @@ -606,11 +604,11 @@ private void setPartitionColumnStatistics( .map(entry -> createMetastoreColumnStatistics(entry.getKey(), columns.get(entry.getKey()), entry.getValue(), rowCount)) .collect(toImmutableList()); if (!metastoreColumnStatistics.isEmpty()) { - setPartitionColumnStatistics(identity, databaseName, tableName, partitionName, metastoreColumnStatistics); + setPartitionColumnStatistics(databaseName, tableName, partitionName, metastoreColumnStatistics); } } - private void setPartitionColumnStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, List statistics) + private void setPartitionColumnStatistics(String databaseName, String tableName, String partitionName, List statistics) { try { retry() @@ -618,7 +616,6 @@ private void setPartitionColumnStatistics(HiveIdentity identity, String database .stopOnIllegalExceptions() .run("setPartitionColumnStatistics", stats.getSetPartitionColumnStatistics().wrap(() -> { setColumnStatistics( - identity, format("partition of table %s.%s", databaseName, tableName), statistics, (client, stats) -> client.setPartitionColumnStatistics(databaseName, tableName, partitionName, stats)); @@ -636,14 +633,14 @@ private void setPartitionColumnStatistics(HiveIdentity identity, String database } } - private void deletePartitionColumnStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, String columnName) + private void deletePartitionColumnStatistics(String databaseName, String tableName, String partitionName, String columnName) { try { retry() .stopOn(NoSuchObjectException.class, InvalidObjectException.class, MetaException.class, InvalidInputException.class) .stopOnIllegalExceptions() .run("deletePartitionColumnStatistics", stats.getDeletePartitionColumnStatistics().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.deletePartitionColumnStatistics(databaseName, tableName, partitionName, columnName); } return null; @@ -660,7 +657,7 @@ private void deletePartitionColumnStatistics(HiveIdentity identity, String datab } } - private void setColumnStatistics(HiveIdentity identity, String objectName, List statistics, Call1> saveColumnStatistics) + private void setColumnStatistics(String objectName, List statistics, Call1> saveColumnStatistics) throws TException { boolean containsDateStatistics = statistics.stream().anyMatch(stats -> stats.getStatsData().isSetDateStats()); @@ -675,7 +672,7 @@ private void setColumnStatistics(HiveIdentity identity, String objectName, List< } if (!containsDateStatistics || metastoreSupportsDateStatistics.equals(Optional.of(TRUE))) { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { saveColumnStatistics.call(client, statistics); } return; @@ -691,7 +688,7 @@ private void setColumnStatistics(HiveIdentity identity, String objectName, List< verify(!dateStatistics.isEmpty() && metastoreSupportsDateStatistics.equals(Optional.empty())); - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { saveColumnStatistics.call(client, statisticsExceptDate); try { @@ -711,14 +708,14 @@ private void setColumnStatistics(HiveIdentity identity, String objectName, List< } @Override - public void createRole(HiveIdentity identity, String role, String grantor) + public void createRole(String role, String grantor) { try { retry() .stopOn(MetaException.class) .stopOnIllegalExceptions() .run("createRole", stats.getCreateRole().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.createRole(role, grantor); return null; } @@ -733,14 +730,14 @@ public void createRole(HiveIdentity identity, String role, String grantor) } @Override - public void dropRole(HiveIdentity identity, String role) + public void dropRole(String role) { try { retry() .stopOn(MetaException.class) .stopOnIllegalExceptions() .run("dropRole", stats.getDropRole().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.dropRole(role); return null; } @@ -755,14 +752,14 @@ public void dropRole(HiveIdentity identity, String role) } @Override - public Set listRoles(HiveIdentity identity) + public Set listRoles() { try { return retry() .stopOn(MetaException.class) .stopOnIllegalExceptions() .run("listRoles", stats.getListRoles().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return ImmutableSet.copyOf(client.getRoleNames()); } })); @@ -776,12 +773,11 @@ public Set listRoles(HiveIdentity identity) } @Override - public void grantRoles(HiveIdentity identity, Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) + public void grantRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) { for (HivePrincipal grantee : grantees) { for (String role : roles) { grantRole( - identity, role, grantee.getName(), fromTrinoPrincipalType(grantee.getType()), grantor.getName(), fromTrinoPrincipalType(grantor.getType()), @@ -790,14 +786,14 @@ public void grantRoles(HiveIdentity identity, Set roles, Set { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.grantRole(role, granteeName, granteeType, grantorName, grantorType, grantOption); return null; } @@ -812,12 +808,11 @@ private void grantRole(HiveIdentity identity, String role, String granteeName, P } @Override - public void revokeRoles(HiveIdentity identity, Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) + public void revokeRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) { for (HivePrincipal grantee : grantees) { for (String role : roles) { revokeRole( - identity, role, grantee.getName(), fromTrinoPrincipalType(grantee.getType()), adminOption); @@ -825,14 +820,14 @@ public void revokeRoles(HiveIdentity identity, Set roles, Set { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.revokeRole(role, granteeName, granteeType, grantOption); return null; } @@ -847,14 +842,14 @@ private void revokeRole(HiveIdentity identity, String role, String granteeName, } @Override - public Set listGrantedPrincipals(HiveIdentity identity, String role) + public Set listGrantedPrincipals(String role) { try { return retry() .stopOn(MetaException.class) .stopOnIllegalExceptions() .run("listPrincipals", stats.getListGrantedPrincipals().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return fromRolePrincipalGrants(client.listGrantedPrincipals(role)); } })); @@ -868,14 +863,14 @@ public Set listGrantedPrincipals(HiveIdentity identity, String role) } @Override - public Set listRoleGrants(HiveIdentity identity, HivePrincipal principal) + public Set listRoleGrants(HivePrincipal principal) { try { return retry() .stopOn(MetaException.class) .stopOnIllegalExceptions() .run("listRoleGrants", stats.getListRoleGrants().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return fromRolePrincipalGrants(client.listRoleGrants(principal.getName(), fromTrinoPrincipalType(principal.getType()))); } })); @@ -889,7 +884,7 @@ public Set listRoleGrants(HiveIdentity identity, HivePrincipal princi } @Override - public List getAllViews(HiveIdentity identity, String databaseName) + public List getAllViews(String databaseName) { try { return retry() @@ -898,14 +893,14 @@ public List getAllViews(HiveIdentity identity, String databaseName) .run("getAllViews", stats.getGetAllViews().wrap(() -> { if (translateHiveViews) { return alternativeCall( - () -> createMetastoreClient(identity), + this::createMetastoreClient, exception -> !isUnknownMethodExceptionalResponse(exception), chosesGetAllViewsAlternative, client -> client.getTableNamesByType(databaseName, TableType.VIRTUAL_VIEW.name()), // fallback to enumerating Presto views only (Hive views will still be executed, but will be listed as tables) - client -> doGetTablesWithParameter(identity, databaseName, PRESTO_VIEW_FLAG, "true")); + client -> doGetTablesWithParameter(databaseName, PRESTO_VIEW_FLAG, "true")); } - return doGetTablesWithParameter(identity, databaseName, PRESTO_VIEW_FLAG, "true"); + return doGetTablesWithParameter(databaseName, PRESTO_VIEW_FLAG, "true"); })); } catch (UnknownDBException e) { @@ -919,7 +914,7 @@ public List getAllViews(HiveIdentity identity, String databaseName) } } - private List doGetTablesWithParameter(HiveIdentity identity, String databaseName, String parameterKey, String parameterValue) + private List doGetTablesWithParameter(String databaseName, String parameterKey, String parameterValue) throws TException { checkArgument(TABLE_PARAMETER_SAFE_KEY_PATTERN.matcher(parameterKey).matches(), "Parameter key contains invalid characters: '%s'", parameterKey); @@ -940,7 +935,7 @@ private List doGetTablesWithParameter(HiveIdentity identity, String data String filterWithLike = HIVE_FILTER_FIELD_PARAMS + parameterKey + " LIKE \"" + parameterValue + "\""; return alternativeCall( - () -> createMetastoreClient(identity), + this::createMetastoreClient, ThriftHiveMetastore::defaultIsValidExceptionalResponse, chosenTableParamAlternative, client -> client.getTableNamesByFilter(databaseName, filterWithEquals), @@ -948,14 +943,14 @@ private List doGetTablesWithParameter(HiveIdentity identity, String data } @Override - public void createDatabase(HiveIdentity identity, Database database) + public void createDatabase(Database database) { try { retry() .stopOn(AlreadyExistsException.class, InvalidObjectException.class, MetaException.class) .stopOnIllegalExceptions() .run("createDatabase", stats.getCreateDatabase().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.createDatabase(database); } return null; @@ -973,14 +968,14 @@ public void createDatabase(HiveIdentity identity, Database database) } @Override - public void dropDatabase(HiveIdentity identity, String databaseName, boolean deleteData) + public void dropDatabase(String databaseName, boolean deleteData) { try { retry() .stopOn(NoSuchObjectException.class, InvalidOperationException.class) .stopOnIllegalExceptions() .run("dropDatabase", stats.getDropDatabase().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.dropDatabase(databaseName, deleteData, false); } return null; @@ -998,14 +993,14 @@ public void dropDatabase(HiveIdentity identity, String databaseName, boolean del } @Override - public void alterDatabase(HiveIdentity identity, String databaseName, Database database) + public void alterDatabase(String databaseName, Database database) { try { retry() .stopOn(NoSuchObjectException.class, MetaException.class) .stopOnIllegalExceptions() .run("alterDatabase", stats.getAlterDatabase().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.alterDatabase(databaseName, database); } return null; @@ -1023,14 +1018,14 @@ public void alterDatabase(HiveIdentity identity, String databaseName, Database d } @Override - public void createTable(HiveIdentity identity, Table table) + public void createTable(Table table) { try { retry() .stopOn(AlreadyExistsException.class, InvalidObjectException.class, MetaException.class, NoSuchObjectException.class) .stopOnIllegalExceptions() .run("createTable", stats.getCreateTable().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.createTable(table); } return null; @@ -1051,14 +1046,14 @@ public void createTable(HiveIdentity identity, Table table) } @Override - public void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) + public void dropTable(String databaseName, String tableName, boolean deleteData) { try { retry() .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("dropTable", stats.getDropTable().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { Table table = client.getTable(databaseName, tableName); client.dropTable(databaseName, tableName, deleteData); String tableLocation = table.getSd().getLocation(); @@ -1097,14 +1092,14 @@ private static boolean isManagedTable(Table table) } @Override - public void alterTable(HiveIdentity identity, String databaseName, String tableName, Table table) + public void alterTable(String databaseName, String tableName, Table table) { try { retry() .stopOn(InvalidOperationException.class, MetaException.class) .stopOnIllegalExceptions() .run("alterTable", stats.getAlterTable().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { EnvironmentContext context = new EnvironmentContext(); // This prevents Hive 3.x from collecting basic table stats at table creation time. // These stats are not useful by themselves and can take very long time to collect when creating an @@ -1127,14 +1122,14 @@ public void alterTable(HiveIdentity identity, String databaseName, String tableN } @Override - public void alterTransactionalTable(HiveIdentity identity, Table table, long transactionId, long writeId) + public void alterTransactionalTable(Table table, long transactionId, long writeId) { try { retry() .stopOn(InvalidOperationException.class, MetaException.class) .stopOnIllegalExceptions() .run("alterTransactionalTable", stats.getAlterTransactionalTable().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.alterTransactionalTable(table, transactionId, writeId, new EnvironmentContext()); } return null; @@ -1152,7 +1147,7 @@ public void alterTransactionalTable(HiveIdentity identity, Table table, long tra } @Override - public Optional> getPartitionNamesByFilter(HiveIdentity identity, String databaseName, String tableName, List columnNames, TupleDomain partitionKeysFilter) + public Optional> getPartitionNamesByFilter(String databaseName, String tableName, List columnNames, TupleDomain partitionKeysFilter) { checkArgument(!columnNames.isEmpty() || partitionKeysFilter.isAll(), "must pass in all columnNames or the filter must be all"); @@ -1167,7 +1162,7 @@ public Optional> getPartitionNamesByFilter(HiveIdentity identity, S .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getPartitionNames", stats.getGetPartitionNames().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return Optional.of(client.getPartitionNames(databaseName, tableName)); } })); @@ -1176,7 +1171,7 @@ public Optional> getPartitionNamesByFilter(HiveIdentity identity, S .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getPartitionNamesByParts", stats.getGetPartitionNamesByParts().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return Optional.of(client.getPartitionNamesFiltered(databaseName, tableName, parts.get())); } })); @@ -1193,18 +1188,18 @@ public Optional> getPartitionNamesByFilter(HiveIdentity identity, S } @Override - public void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitionsWithStatistics) + public void addPartitions(String databaseName, String tableName, List partitionsWithStatistics) { List partitions = partitionsWithStatistics.stream() .map(ThriftMetastoreUtil::toMetastoreApiPartition) .collect(toImmutableList()); - addPartitionsWithoutStatistics(identity, databaseName, tableName, partitions); + addPartitionsWithoutStatistics(databaseName, tableName, partitions); for (PartitionWithStatistics partitionWithStatistics : partitionsWithStatistics) { - storePartitionColumnStatistics(identity, databaseName, tableName, partitionWithStatistics.getPartitionName(), partitionWithStatistics); + storePartitionColumnStatistics(databaseName, tableName, partitionWithStatistics.getPartitionName(), partitionWithStatistics); } } - private void addPartitionsWithoutStatistics(HiveIdentity identity, String databaseName, String tableName, List partitions) + private void addPartitionsWithoutStatistics(String databaseName, String tableName, List partitions) { if (partitions.isEmpty()) { return; @@ -1214,7 +1209,7 @@ private void addPartitionsWithoutStatistics(HiveIdentity identity, String databa .stopOn(AlreadyExistsException.class, InvalidObjectException.class, MetaException.class, NoSuchObjectException.class, TrinoException.class) .stopOnIllegalExceptions() .run("addPartitions", stats.getAddPartitions().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { int partitionsAdded = client.addPartitions(partitions); if (partitionsAdded != partitions.size()) { throw new TrinoException(HIVE_METASTORE_ERROR, @@ -1239,14 +1234,14 @@ private void addPartitionsWithoutStatistics(HiveIdentity identity, String databa } @Override - public void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData) + public void dropPartition(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(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.dropPartition(databaseName, tableName, parts, deleteData); } return null; @@ -1264,21 +1259,21 @@ public void dropPartition(HiveIdentity identity, String databaseName, String tab } @Override - public void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partitionWithStatistics) + public void alterPartition(String databaseName, String tableName, PartitionWithStatistics partitionWithStatistics) { - alterPartitionWithoutStatistics(identity, databaseName, tableName, toMetastoreApiPartition(partitionWithStatistics)); - storePartitionColumnStatistics(identity, databaseName, tableName, partitionWithStatistics.getPartitionName(), partitionWithStatistics); - dropExtraColumnStatisticsAfterAlterPartition(identity, databaseName, tableName, partitionWithStatistics); + alterPartitionWithoutStatistics(databaseName, tableName, toMetastoreApiPartition(partitionWithStatistics)); + storePartitionColumnStatistics(databaseName, tableName, partitionWithStatistics.getPartitionName(), partitionWithStatistics); + dropExtraColumnStatisticsAfterAlterPartition(databaseName, tableName, partitionWithStatistics); } - private void alterPartitionWithoutStatistics(HiveIdentity identity, String databaseName, String tableName, Partition partition) + private void alterPartitionWithoutStatistics(String databaseName, String tableName, Partition partition) { try { retry() .stopOn(NoSuchObjectException.class, MetaException.class) .stopOnIllegalExceptions() .run("alterPartition", stats.getAlterPartition().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { client.alterPartition(databaseName, tableName, partition); } return null; @@ -1295,7 +1290,7 @@ private void alterPartitionWithoutStatistics(HiveIdentity identity, String datab } } - private void storePartitionColumnStatistics(HiveIdentity identity, String databaseName, String tableName, String partitionName, PartitionWithStatistics partitionWithStatistics) + private void storePartitionColumnStatistics(String databaseName, String tableName, String partitionName, PartitionWithStatistics partitionWithStatistics) { PartitionStatistics statistics = partitionWithStatistics.getStatistics(); Map columnStatistics = statistics.getColumnStatistics(); @@ -1304,7 +1299,7 @@ private void storePartitionColumnStatistics(HiveIdentity identity, String databa } Map columnTypes = partitionWithStatistics.getPartition().getColumns().stream() .collect(toImmutableMap(Column::getName, Column::getType)); - setPartitionColumnStatistics(identity, databaseName, tableName, partitionName, columnTypes, columnStatistics, statistics.getBasicStatistics().getRowCount()); + setPartitionColumnStatistics(databaseName, tableName, partitionName, columnTypes, columnStatistics, statistics.getBasicStatistics().getRowCount()); } /* @@ -1316,7 +1311,6 @@ private void storePartitionColumnStatistics(HiveIdentity identity, String databa * 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) @@ -1338,7 +1332,6 @@ 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), @@ -1346,12 +1339,12 @@ private void dropExtraColumnStatisticsAfterAlterPartition( .getOrDefault(partitionName, ImmutableList.of()); for (ColumnStatisticsObj statistics : statisticsToBeRemoved) { - deletePartitionColumnStatistics(identity, databaseName, tableName, partitionName, statistics.getColName()); + deletePartitionColumnStatistics(databaseName, tableName, partitionName, statistics.getColName()); } } @Override - public Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues) + public Optional getPartition(String databaseName, String tableName, List partitionValues) { requireNonNull(partitionValues, "partitionValues is null"); try { @@ -1359,7 +1352,7 @@ public Optional getPartition(HiveIdentity identity, String databaseNa .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getPartition", stats.getGetPartition().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return Optional.of(client.getPartition(databaseName, tableName, partitionValues)); } })); @@ -1376,7 +1369,7 @@ public Optional getPartition(HiveIdentity identity, String databaseNa } @Override - public List getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames) + public List getPartitionsByNames(String databaseName, String tableName, List partitionNames) { requireNonNull(partitionNames, "partitionNames is null"); checkArgument(!partitionNames.isEmpty(), "partitionNames is empty"); @@ -1386,7 +1379,7 @@ public List getPartitionsByNames(HiveIdentity identity, String databa .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("getPartitionsByNames", stats.getGetPartitionsByNames().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { return client.getPartitionsByNames(databaseName, tableName, partitionNames); } })); @@ -1404,7 +1397,7 @@ public List getPartitionsByNames(HiveIdentity identity, String databa } @Override - public void grantTablePrivileges(HiveIdentity identity, String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) + public void grantTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) { Set requestedPrivileges = privileges.stream() .map(privilege -> new HivePrivilegeInfo(privilege, grantOption, grantor, grantee)) @@ -1416,8 +1409,8 @@ public void grantTablePrivileges(HiveIdentity identity, String databaseName, Str retry() .stopOnIllegalExceptions() .run("grantTablePrivileges", stats.getGrantTablePrivileges().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { - Set existingPrivileges = listTablePrivileges(identity, databaseName, tableName, Optional.of(tableOwner), Optional.of(grantee)); + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { + Set existingPrivileges = listTablePrivileges(databaseName, tableName, Optional.of(tableOwner), Optional.of(grantee)); Set privilegesToGrant = new HashSet<>(requestedPrivileges); Iterator iterator = privilegesToGrant.iterator(); @@ -1456,7 +1449,7 @@ else if (existingPrivilege.isContainedIn(requestedPrivilege)) { } @Override - public void revokeTablePrivileges(HiveIdentity identity, String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) + public void revokeTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) { Set requestedPrivileges = privileges.stream() .map(privilege -> new HivePrivilegeInfo(privilege, grantOption, grantor, grantee)) @@ -1468,8 +1461,8 @@ public void revokeTablePrivileges(HiveIdentity identity, String databaseName, St retry() .stopOnIllegalExceptions() .run("revokeTablePrivileges", stats.getRevokeTablePrivileges().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { - Set existingHivePrivileges = listTablePrivileges(identity, databaseName, tableName, Optional.of(tableOwner), Optional.of(grantee)).stream() + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { + Set existingHivePrivileges = listTablePrivileges(databaseName, tableName, Optional.of(tableOwner), Optional.of(grantee)).stream() .map(HivePrivilegeInfo::getHivePrivilege) .collect(toImmutableSet()); @@ -1495,13 +1488,13 @@ public void revokeTablePrivileges(HiveIdentity identity, String databaseName, St } @Override - public Set listTablePrivileges(HiveIdentity identity, String databaseName, String tableName, Optional tableOwner, Optional principal) + public Set listTablePrivileges(String databaseName, String tableName, Optional tableOwner, Optional principal) { try { return retry() .stopOnIllegalExceptions() .run("listTablePrivileges", stats.getListTablePrivileges().wrap(() -> { - try (ThriftMetastoreClient client = createMetastoreClient(identity)) { + try (ThriftMetastoreClient client = createMetastoreClient()) { ImmutableSet.Builder privileges = ImmutableSet.builder(); List hiveObjectPrivilegeList; if (principal.isEmpty()) { @@ -1537,14 +1530,14 @@ public Set listTablePrivileges(HiveIdentity identity, String } @Override - public long openTransaction(HiveIdentity identity, AcidTransactionOwner transactionOwner) + public long openTransaction(AcidTransactionOwner transactionOwner) { requireNonNull(transactionOwner, "transactionOwner is null"); try { return retry() .stopOnIllegalExceptions() .run("openTransaction", stats.getOpenTransaction().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { return metastoreClient.openTransaction(transactionOwner.toString()); } })); @@ -1558,13 +1551,13 @@ public long openTransaction(HiveIdentity identity, AcidTransactionOwner transact } @Override - public void commitTransaction(HiveIdentity identity, long transactionId) + public void commitTransaction(long transactionId) { try { retry() .stopOnIllegalExceptions() .run("commitTransaction", stats.getCommitTransaction().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { metastoreClient.commitTransaction(transactionId); } return null; @@ -1579,13 +1572,13 @@ public void commitTransaction(HiveIdentity identity, long transactionId) } @Override - public void abortTransaction(HiveIdentity identity, long transactionId) + public void abortTransaction(long transactionId) { try { retry() .stopOnIllegalExceptions() .run("abortTransaction", stats.getAbortTransaction().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { metastoreClient.abortTransaction(transactionId); } return null; @@ -1600,13 +1593,13 @@ public void abortTransaction(HiveIdentity identity, long transactionId) } @Override - public void sendTransactionHeartbeat(HiveIdentity identity, long transactionId) + public void sendTransactionHeartbeat(long transactionId) { try { retry() .stopOnIllegalExceptions() .run("sendTransactionHeartbeat", (() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { metastoreClient.sendTransactionHeartbeat(transactionId); } return null; @@ -1622,19 +1615,17 @@ public void sendTransactionHeartbeat(HiveIdentity identity, long transactionId) @Override public void acquireSharedReadLock( - HiveIdentity identity, AcidTransactionOwner transactionOwner, String queryId, long transactionId, List fullTables, List partitions) { - acquireSharedLock(identity, transactionOwner, queryId, transactionId, fullTables, partitions, DataOperationType.SELECT, false); + acquireSharedLock(transactionOwner, queryId, transactionId, fullTables, partitions, DataOperationType.SELECT, false); } @Override public void acquireTableWriteLock( - HiveIdentity identity, AcidTransactionOwner transactionOwner, String queryId, long transactionId, @@ -1643,11 +1634,10 @@ public void acquireTableWriteLock( DataOperationType operation, boolean isDynamicPartitionWrite) { - acquireSharedLock(identity, transactionOwner, queryId, transactionId, ImmutableList.of(new SchemaTableName(dbName, tableName)), Collections.emptyList(), operation, isDynamicPartitionWrite); + acquireSharedLock(transactionOwner, queryId, transactionId, ImmutableList.of(new SchemaTableName(dbName, tableName)), Collections.emptyList(), operation, isDynamicPartitionWrite); } private void acquireSharedLock( - HiveIdentity identity, AcidTransactionOwner transactionOwner, String queryId, long transactionId, @@ -1676,12 +1666,11 @@ private void acquireSharedLock( request.addLockComponent(createLockComponentForOperation(partition.getTableName(), operation, isDynamicPartitionWrite, Optional.of(partition.getPartitionId()))); } - acquireLock(identity, format("hive transaction %s for query %s", transactionId, queryId), request.build()); + acquireLock(format("hive transaction %s for query %s", transactionId, queryId), request.build()); } @Override public long acquireTableExclusiveLock( - HiveIdentity identity, AcidTransactionOwner transactionOwner, String queryId, String dbName, @@ -1694,16 +1683,16 @@ public long acquireTableExclusiveLock( .addLockComponent(lockComponent) .setUser(transactionOwner.toString()) .build(); - return acquireLock(identity, format("query %s", queryId), lockRequest); + return acquireLock(format("query %s", queryId), lockRequest); } - private long acquireLock(HiveIdentity identity, String context, LockRequest lockRequest) + private long acquireLock(String context, LockRequest lockRequest) { try { LockResponse response = retry() .stopOn(NoSuchTxnException.class, TxnAbortedException.class, MetaException.class) .run("acquireLock", stats.getAcquireLock().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { return metastoreClient.acquireLock(lockRequest); } })); @@ -1713,7 +1702,7 @@ private long acquireLock(HiveIdentity identity, String context, LockRequest lock while (response.getState() == LockState.WAITING) { if (Duration.nanosSince(waitStart).compareTo(maxWaitForLock) > 0) { // timed out - throw unlockSuppressing(identity, lockId, new TrinoException(HIVE_TABLE_LOCK_NOT_ACQUIRED, format("Timed out waiting for lock %d for %s", lockId, context))); + throw unlockSuppressing(lockId, new TrinoException(HIVE_TABLE_LOCK_NOT_ACQUIRED, format("Timed out waiting for lock %d for %s", lockId, context))); } log.debug("Waiting for lock %d for %s", lockId, context); @@ -1721,14 +1710,14 @@ private long acquireLock(HiveIdentity identity, String context, LockRequest lock response = retry() .stopOn(NoSuchTxnException.class, NoSuchLockException.class, TxnAbortedException.class, MetaException.class) .run("checkLock", stats.getCheckLock().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { return metastoreClient.checkLock(lockId); } })); } if (response.getState() != LockState.ACQUIRED) { - throw unlockSuppressing(identity, lockId, new TrinoException(HIVE_TABLE_LOCK_NOT_ACQUIRED, "Could not acquire lock. Lock in state " + response.getState())); + throw unlockSuppressing(lockId, new TrinoException(HIVE_TABLE_LOCK_NOT_ACQUIRED, "Could not acquire lock. Lock in state " + response.getState())); } return response.getLockid(); @@ -1741,10 +1730,10 @@ private long acquireLock(HiveIdentity identity, String context, LockRequest lock } } - private T unlockSuppressing(HiveIdentity identity, long lockId, T exception) + private T unlockSuppressing(long lockId, T exception) { try { - releaseTableLock(identity, lockId); + releaseTableLock(lockId); } catch (RuntimeException e) { exception.addSuppressed(e); @@ -1753,13 +1742,13 @@ private T unlockSuppressing(HiveIdentity identity, long lo } @Override - public void releaseTableLock(HiveIdentity identity, long lockId) + public void releaseTableLock(long lockId) { try { retry() .stopOn(NoSuchTxnException.class, NoSuchLockException.class, TxnAbortedException.class, MetaException.class) .run("unlock", stats.getUnlock().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { metastoreClient.unlock(lockId); } return null; @@ -1793,13 +1782,13 @@ private static LockComponent createLockComponentForOperation(SchemaTableName tab } @Override - public String getValidWriteIds(HiveIdentity identity, List tables, long currentTransactionId) + public String getValidWriteIds(List tables, long currentTransactionId) { try { return retry() .stopOnIllegalExceptions() .run("getValidWriteIds", stats.getValidWriteIds().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { return metastoreClient.getValidWriteIds( tables.stream() .map(table -> format("%s.%s", table.getSchemaName(), table.getTableName())) @@ -1819,13 +1808,13 @@ public String getValidWriteIds(HiveIdentity identity, List tabl } @Override - public Optional getConfigValue(HiveIdentity identity, String name) + public Optional getConfigValue(String name) { try { return retry() .stopOnIllegalExceptions() .run("getConfigValueFromServer", () -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { return Optional.ofNullable(metastoreClient.getConfigValue(name, null)); } }); @@ -1843,13 +1832,13 @@ public Optional getConfigValue(HiveIdentity identity, String name) } @Override - public long allocateWriteId(HiveIdentity identity, String dbName, String tableName, long transactionId) + public long allocateWriteId(String dbName, String tableName, long transactionId) { try { return retry() .stopOnIllegalExceptions() .run("allocateWriteId", stats.getAllocateWriteId().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { List list = metastoreClient.allocateTableWriteIds(dbName, tableName, ImmutableList.of(transactionId)); return getOnlyElement(list).getWriteId(); } @@ -1864,7 +1853,7 @@ public long allocateWriteId(HiveIdentity identity, String dbName, String tableNa } @Override - public void updateTableWriteId(HiveIdentity identity, String dbName, String tableName, long transactionId, long writeId, OptionalLong rowCountChange) + public void updateTableWriteId(String dbName, String tableName, long transactionId, long writeId, OptionalLong rowCountChange) { checkArgument(transactionId > 0, "transactionId should be a positive integer, but was %s", transactionId); requireNonNull(dbName, "dbName is null"); @@ -1874,7 +1863,7 @@ public void updateTableWriteId(HiveIdentity identity, String dbName, String tabl retry() .stopOnIllegalExceptions() .run("updateTableWriteId", stats.getUpdateTableWriteId().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { metastoreClient.updateTableWriteId(dbName, tableName, transactionId, writeId, rowCountChange); } return null; @@ -1889,14 +1878,14 @@ public void updateTableWriteId(HiveIdentity identity, String dbName, String tabl } @Override - public void alterPartitions(HiveIdentity identity, String dbName, String tableName, List partitions, long writeId) + public void alterPartitions(String dbName, String tableName, List partitions, long writeId) { checkArgument(writeId > 0, "writeId should be a positive integer, but was %s", writeId); try { retry() .stopOnIllegalExceptions() .run("alterPartitions", stats.getAlterPartitions().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { metastoreClient.alterPartitions(dbName, tableName, partitions, writeId); } return null; @@ -1911,7 +1900,7 @@ public void alterPartitions(HiveIdentity identity, String dbName, String tableNa } @Override - public void addDynamicPartitions(HiveIdentity identity, String dbName, String tableName, List partitionNames, long transactionId, long writeId, AcidOperation operation) + public void addDynamicPartitions(String dbName, String tableName, List partitionNames, long transactionId, long writeId, AcidOperation operation) { checkArgument(writeId > 0, "writeId should be a positive integer, but was %s", writeId); requireNonNull(partitionNames, "partitionNames is null"); @@ -1920,7 +1909,7 @@ public void addDynamicPartitions(HiveIdentity identity, String dbName, String ta retry() .stopOnIllegalExceptions() .run("alterPartitions", stats.getAddDynamicPartitions().wrap(() -> { - try (ThriftMetastoreClient metastoreClient = createMetastoreClient(identity)) { + try (ThriftMetastoreClient metastoreClient = createMetastoreClient()) { metastoreClient.addDynamicPartitions(dbName, tableName, partitionNames, transactionId, writeId, operation); } return null; @@ -2031,7 +2020,7 @@ private static boolean isUnknownMethodExceptionalResponse(Exception exception) return applicationException.getType() == UNKNOWN_METHOD; } - private ThriftMetastoreClient createMetastoreClient(HiveIdentity identity) + private ThriftMetastoreClient createMetastoreClient() throws TException { return metastoreFactory.createMetastoreClient(identity); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastoreFactory.java index 2d1cab174f10..632c5b533a06 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastoreFactory.java @@ -16,11 +16,14 @@ import io.airlift.units.Duration; import io.trino.plugin.hive.HdfsEnvironment; import io.trino.plugin.hive.HideDeltaLakeTables; +import io.trino.spi.security.ConnectorIdentity; import org.weakref.jmx.Flatten; import org.weakref.jmx.Managed; import javax.inject.Inject; +import java.util.Optional; + import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; @@ -79,9 +82,10 @@ public boolean isImpersonationEnabled() } @Override - public ThriftMetastore createMetastore() + public ThriftMetastore createMetastore(Optional identity) { return new ThriftHiveMetastore( + identity, hdfsEnvironment, metastoreFactory, backoffScaleFactor, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastore.java index 3e978045fa16..0698fb3fe78f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastore.java @@ -17,7 +17,6 @@ import io.trino.plugin.hive.PartitionStatistics; import io.trino.plugin.hive.acid.AcidOperation; import io.trino.plugin.hive.acid.AcidTransaction; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.AcidTransactionOwner; import io.trino.plugin.hive.metastore.HivePrincipal; import io.trino.plugin.hive.metastore.HivePrivilegeInfo; @@ -47,81 +46,81 @@ public interface ThriftMetastore { - void createDatabase(HiveIdentity identity, Database database); + void createDatabase(Database database); - void dropDatabase(HiveIdentity identity, String databaseName, boolean deleteData); + void dropDatabase(String databaseName, boolean deleteData); - void alterDatabase(HiveIdentity identity, String databaseName, Database database); + void alterDatabase(String databaseName, Database database); - void createTable(HiveIdentity identity, Table table); + void createTable(Table table); - void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData); + void dropTable(String databaseName, String tableName, boolean deleteData); - void alterTable(HiveIdentity identity, String databaseName, String tableName, Table table); + void alterTable(String databaseName, String tableName, Table table); - void alterTransactionalTable(HiveIdentity identity, Table table, long transactionId, long writeId); + void alterTransactionalTable(Table table, long transactionId, long writeId); - List getAllDatabases(HiveIdentity identity); + List getAllDatabases(); - List getAllTables(HiveIdentity identity, String databaseName); + List getAllTables(String databaseName); - List getTablesWithParameter(HiveIdentity identity, String databaseName, String parameterKey, String parameterValue); + List getTablesWithParameter(String databaseName, String parameterKey, String parameterValue); - List getAllViews(HiveIdentity identity, String databaseName); + List getAllViews(String databaseName); - Optional getDatabase(HiveIdentity identity, String databaseName); + Optional getDatabase(String databaseName); - void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitions); + void addPartitions(String databaseName, String tableName, List partitions); - void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData); + void dropPartition(String databaseName, String tableName, List parts, boolean deleteData); - void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partition); + void alterPartition(String databaseName, String tableName, PartitionWithStatistics partition); - Optional> getPartitionNamesByFilter(HiveIdentity identity, String databaseName, String tableName, List columnNames, TupleDomain partitionKeysFilter); + Optional> getPartitionNamesByFilter(String databaseName, String tableName, List columnNames, TupleDomain partitionKeysFilter); - Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues); + Optional getPartition(String databaseName, String tableName, List partitionValues); - List getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames); + List getPartitionsByNames(String databaseName, String tableName, List partitionNames); - Optional
getTable(HiveIdentity identity, String databaseName, String tableName); + Optional
getTable(String databaseName, String tableName); Set getSupportedColumnStatistics(Type type); - PartitionStatistics getTableStatistics(HiveIdentity identity, Table table); + PartitionStatistics getTableStatistics(Table table); - Map getPartitionStatistics(HiveIdentity identity, Table table, List partitions); + Map getPartitionStatistics(Table table, List partitions); - void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, AcidTransaction transaction, Function update); + void updateTableStatistics(String databaseName, String tableName, AcidTransaction transaction, Function update); - void updatePartitionStatistics(HiveIdentity identity, Table table, String partitionName, Function update); + void updatePartitionStatistics(Table table, String partitionName, Function update); - void createRole(HiveIdentity identity, String role, String grantor); + void createRole(String role, String grantor); - void dropRole(HiveIdentity identity, String role); + void dropRole(String role); - Set listRoles(HiveIdentity identity); + Set listRoles(); - void grantRoles(HiveIdentity identity, Set roles, Set grantees, boolean adminOption, HivePrincipal grantor); + void grantRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor); - void revokeRoles(HiveIdentity identity, Set roles, Set grantees, boolean adminOption, HivePrincipal grantor); + void revokeRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor); - Set listGrantedPrincipals(HiveIdentity identity, String role); + Set listGrantedPrincipals(String role); - Set listRoleGrants(HiveIdentity identity, HivePrincipal principal); + Set listRoleGrants(HivePrincipal principal); - void grantTablePrivileges(HiveIdentity identity, String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption); + void grantTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption); - void revokeTablePrivileges(HiveIdentity identity, String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption); + void revokeTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption); /** * @param tableOwner * @param principal when empty, all table privileges are returned */ - Set listTablePrivileges(HiveIdentity identity, String databaseName, String tableName, Optional tableOwner, Optional principal); + Set listTablePrivileges(String databaseName, String tableName, Optional tableOwner, Optional principal); - default Optional> getFields(HiveIdentity identity, String databaseName, String tableName) + default Optional> getFields(String databaseName, String tableName) { - Optional
table = getTable(identity, databaseName, tableName); + Optional
table = getTable(databaseName, tableName); if (table.isEmpty()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -133,28 +132,27 @@ default Optional> getFields(HiveIdentity identity, String data return Optional.of(table.get().getSd().getCols()); } - default long openTransaction(HiveIdentity identity, AcidTransactionOwner transactionOwner) + default long openTransaction(AcidTransactionOwner transactionOwner) { throw new UnsupportedOperationException(); } - default void commitTransaction(HiveIdentity identity, long transactionId) + default void commitTransaction(long transactionId) { throw new UnsupportedOperationException(); } - default void abortTransaction(HiveIdentity identity, long transactionId) + default void abortTransaction(long transactionId) { throw new UnsupportedOperationException(); } - default void sendTransactionHeartbeat(HiveIdentity identity, long transactionId) + default void sendTransactionHeartbeat(long transactionId) { throw new UnsupportedOperationException(); } default void acquireSharedReadLock( - HiveIdentity identity, AcidTransactionOwner transactionOwner, String queryId, long transactionId, @@ -164,23 +162,22 @@ default void acquireSharedReadLock( throw new UnsupportedOperationException(); } - default String getValidWriteIds(HiveIdentity identity, List tables, long currentTransactionId) + default String getValidWriteIds(List tables, long currentTransactionId) { throw new UnsupportedOperationException(); } - default Optional getConfigValue(HiveIdentity identity, String name) + default Optional getConfigValue(String name) { return Optional.empty(); } - default long allocateWriteId(HiveIdentity identity, String dbName, String tableName, long transactionId) + default long allocateWriteId(String dbName, String tableName, long transactionId) { throw new UnsupportedOperationException(); } default void acquireTableWriteLock( - HiveIdentity identity, AcidTransactionOwner transactionOwner, String queryId, long transactionId, @@ -193,7 +190,6 @@ default void acquireTableWriteLock( } default long acquireTableExclusiveLock( - HiveIdentity identity, AcidTransactionOwner transactionOwner, String queryId, String dbName, @@ -202,22 +198,22 @@ default long acquireTableExclusiveLock( throw new UnsupportedOperationException(); } - default void releaseTableLock(HiveIdentity identity, long lockId) + default void releaseTableLock(long lockId) { throw new UnsupportedOperationException(); } - default void updateTableWriteId(HiveIdentity identity, String dbName, String tableName, long transactionId, long writeId, OptionalLong rowCountChange) + default void updateTableWriteId(String dbName, String tableName, long transactionId, long writeId, OptionalLong rowCountChange) { throw new UnsupportedOperationException(); } - default void alterPartitions(HiveIdentity identity, String dbName, String tableName, List partitions, long writeId) + default void alterPartitions(String dbName, String tableName, List partitions, long writeId) { throw new UnsupportedOperationException(); } - default void addDynamicPartitions(HiveIdentity identity, String dbName, String tableName, List partitionNames, long transactionId, long writeId, AcidOperation operation) + default void addDynamicPartitions(String dbName, String tableName, List partitionNames, long transactionId, long writeId, AcidOperation operation) { throw new UnsupportedOperationException(); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreFactory.java index 4bae16f295bb..5fe5b88e0cb3 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreFactory.java @@ -13,9 +13,13 @@ */ package io.trino.plugin.hive.metastore.thrift; +import io.trino.spi.security.ConnectorIdentity; + +import java.util.Optional; + public interface ThriftMetastoreFactory { boolean isImpersonationEnabled(); - ThriftMetastore createMetastore(); + ThriftMetastore createMetastore(Optional identity); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/TokenDelegationThriftMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/TokenDelegationThriftMetastoreFactory.java index e6adbe678eb4..72a792dafbd6 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/TokenDelegationThriftMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/TokenDelegationThriftMetastoreFactory.java @@ -18,9 +18,9 @@ import com.google.common.util.concurrent.UncheckedExecutionException; import io.trino.collect.cache.NonEvictableLoadingCache; import io.trino.plugin.hive.HdfsEnvironment; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreAuthenticationConfig.ThriftMetastoreAuthenticationType; import io.trino.spi.TrinoException; +import io.trino.spi.security.ConnectorIdentity; import org.apache.thrift.TException; import javax.inject.Inject; @@ -66,14 +66,15 @@ private ThriftMetastoreClient createMetastoreClient() return clientProvider.createMetastoreClient(Optional.empty()); } - public ThriftMetastoreClient createMetastoreClient(HiveIdentity identity) + public ThriftMetastoreClient createMetastoreClient(Optional identity) throws TException { if (!impersonationEnabled) { return createMetastoreClient(); } - String username = identity.getUsername().orElseThrow(() -> new IllegalStateException("End-user name should exist when metastore impersonation is enabled")); + String username = identity.map(ConnectorIdentity::getUser) + .orElseThrow(() -> new IllegalStateException("End-user name should exist when metastore impersonation is enabled")); if (authenticationEnabled) { String delegationToken; try { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java index 1f47e4eb7c85..f609800c34e2 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java @@ -29,7 +29,6 @@ import io.trino.plugin.base.metrics.LongCount; import io.trino.plugin.hive.HdfsEnvironment.HdfsContext; import io.trino.plugin.hive.LocationService.WriteInfo; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.authentication.NoHdfsAuthentication; import io.trino.plugin.hive.fs.DirectoryLister; import io.trino.plugin.hive.metastore.Column; @@ -785,8 +784,7 @@ protected final void setup(String host, int port, String databaseName, String ti .metastoreClient(HostAndPort.fromParts(host, port)) .hiveConfig(hiveConfig) .hdfsEnvironment(hdfsEnvironment) - .build(), - new HiveIdentity(SESSION.getIdentity())), + .build()), executor, new Duration(1, MINUTES), Optional.of(new Duration(15, SECONDS)), diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java index 3d100c5fe5dd..b3eb88922a1d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java @@ -27,7 +27,6 @@ import io.trino.plugin.hive.AbstractTestHive.HiveTransaction; import io.trino.plugin.hive.AbstractTestHive.Transaction; import io.trino.plugin.hive.HdfsEnvironment.HdfsContext; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.authentication.NoHdfsAuthentication; import io.trino.plugin.hive.fs.FileSystemDirectoryLister; import io.trino.plugin.hive.fs.HiveFileIterator; @@ -189,8 +188,7 @@ protected void setup(String host, int port, String databaseName, boolean s3Selec .metastoreClient(HostAndPort.fromParts(host, port)) .hiveConfig(config) .hdfsEnvironment(hdfsEnvironment) - .build(), - new HiveIdentity(getHiveSession(config).getIdentity())), + .build()), getBasePath(), hdfsEnvironment); locationService = new HiveLocationService(hdfsEnvironment); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveLocal.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveLocal.java index 0f59a9006d05..d9b4356acb11 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveLocal.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveLocal.java @@ -19,7 +19,6 @@ import com.google.common.io.RecursiveDeleteOption; import com.google.common.reflect.ClassPath; import io.airlift.log.Logger; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; @@ -61,7 +60,6 @@ import static io.trino.plugin.hive.HiveType.HIVE_STRING; import static io.trino.plugin.hive.util.HiveBucketing.BucketingVersion.BUCKETING_V1; import static io.trino.plugin.hive.util.HiveUtil.SPARK_TABLE_PROVIDER_KEY; -import static io.trino.testing.TestingConnectorSession.SESSION; import static java.nio.file.Files.copy; import static java.util.Objects.requireNonNull; import static org.testng.Assert.assertEquals; @@ -71,7 +69,6 @@ public abstract class AbstractTestHiveLocal { private static final Logger log = Logger.get(AbstractTestHiveLocal.class); private static final String DEFAULT_TEST_DB_NAME = "test"; - private static final HiveIdentity HIVE_IDENTITY = new HiveIdentity(SESSION.getIdentity()); private File tempDir; private final String testDbName; @@ -86,14 +83,14 @@ protected AbstractTestHiveLocal(String testDbName) this.testDbName = requireNonNull(testDbName, "testDbName is null"); } - protected abstract HiveMetastore createMetastore(File tempDir, HiveIdentity identity); + protected abstract HiveMetastore createMetastore(File tempDir); @BeforeClass(alwaysRun = true) public void initialize() { tempDir = Files.createTempDir(); - HiveMetastore metastore = createMetastore(tempDir, HIVE_IDENTITY); + HiveMetastore metastore = createMetastore(tempDir); metastore.createDatabase( Database.builder() diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseTestHiveOnDataLake.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseTestHiveOnDataLake.java index 6acd3072f1a7..c47c1de505c5 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseTestHiveOnDataLake.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseTestHiveOnDataLake.java @@ -17,7 +17,6 @@ import com.google.common.collect.ImmutableMap; import io.airlift.units.DataSize; import io.trino.Session; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.containers.HiveMinioDataLake; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.Partition; @@ -71,8 +70,7 @@ protected QueryRunner createQueryRunner() this.metastoreClient = new BridgingHiveMetastore( testingThriftHiveMetastoreBuilder() .metastoreClient(this.dockerizedS3DataLake.getHiveHadoop().getHiveMetastoreEndpoint()) - .build(), - HiveIdentity.none()); + .build()); return S3HiveQueryRunner.builder(dockerizedS3DataLake) .setHiveProperties( ImmutableMap.builder() diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileMetastore.java index 2a49be29068a..67c763642c22 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileMetastore.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.hive; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.file.FileHiveMetastore; import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; @@ -30,7 +29,7 @@ public class TestHiveFileMetastore extends AbstractTestHiveLocal { @Override - protected HiveMetastore createMetastore(File tempDir, HiveIdentity identity) + protected HiveMetastore createMetastore(File tempDir) { File baseDir = new File(tempDir, "metastore"); return new FileHiveMetastore( diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveInMemoryMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveInMemoryMetastore.java index c839869600ca..44667f59d5c4 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveInMemoryMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveInMemoryMetastore.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.hive; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; import io.trino.plugin.hive.metastore.thrift.InMemoryThriftMetastore; @@ -29,12 +28,12 @@ public class TestHiveInMemoryMetastore extends AbstractTestHiveLocal { @Override - protected HiveMetastore createMetastore(File tempDir, HiveIdentity identity) + protected HiveMetastore createMetastore(File tempDir) { File baseDir = new File(tempDir, "metastore"); ThriftMetastoreConfig metastoreConfig = new ThriftMetastoreConfig(); InMemoryThriftMetastore hiveMetastore = new InMemoryThriftMetastore(baseDir, metastoreConfig); - return new BridgingHiveMetastore(hiveMetastore, identity); + return new BridgingHiveMetastore(hiveMetastore); } @Test diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java index 3cfb0515f3a4..815889d6e4b3 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java @@ -32,6 +32,8 @@ import io.trino.plugin.hive.s3.HiveS3Config; import io.trino.plugin.hive.s3.TrinoS3ConfigurationInitializer; +import java.util.Optional; + import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; @@ -102,6 +104,6 @@ public ThriftMetastore build() hiveConfig.isTranslateHiveViews(), new ThriftMetastoreConfig(), hdfsEnvironment); - return metastoreFactory.createMetastore(); + return metastoreFactory.createMetastore(Optional.empty()); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java index 33cea081324f..901e60861444 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java @@ -22,7 +22,6 @@ import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.HiveMetastoreClosure; import io.trino.plugin.hive.PartitionStatistics; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HivePrincipal; @@ -92,7 +91,6 @@ import static io.trino.spi.predicate.TupleDomain.withColumnDomains; import static io.trino.spi.security.PrincipalType.USER; import static io.trino.spi.type.VarcharType.VARCHAR; -import static io.trino.testing.TestingConnectorSession.SESSION; import static java.lang.String.format; import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.TimeUnit.SECONDS; @@ -109,7 +107,6 @@ public class TestCachingHiveMetastore { private static final Logger log = Logger.get(TestCachingHiveMetastore.class); - private static final HiveIdentity IDENTITY = new HiveIdentity(SESSION.getIdentity()); private static final PartitionStatistics TEST_STATS = PartitionStatistics.builder() .setColumnStatistics(ImmutableMap.of(TEST_COLUMN, createIntegerColumnStatistics(OptionalLong.empty(), OptionalLong.empty(), OptionalLong.empty(), OptionalLong.empty()))) .build(); @@ -126,7 +123,7 @@ public void setUp() ThriftMetastore thriftHiveMetastore = createThriftHiveMetastore(); executor = listeningDecorator(newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s"))); metastore = cachingHiveMetastore( - new BridgingHiveMetastore(thriftHiveMetastore, IDENTITY), + new BridgingHiveMetastore(thriftHiveMetastore), executor, new Duration(5, TimeUnit.MINUTES), Optional.of(new Duration(1, TimeUnit.MINUTES)), @@ -587,7 +584,7 @@ public void testCachingHiveMetastoreCreationViaMemoize() { ThriftMetastore thriftHiveMetastore = createThriftHiveMetastore(); metastore = memoizeMetastore( - new BridgingHiveMetastore(thriftHiveMetastore, IDENTITY), + new BridgingHiveMetastore(thriftHiveMetastore), 1000); assertEquals(mockClient.getAccessCount(), 0); @@ -784,7 +781,7 @@ private PartitionCachingAssertions() { thriftClient = new MockThriftMetastoreClient(); cachingHiveMetastore = (CachingHiveMetastore) cachingHiveMetastore( - new BridgingHiveMetastore(createThriftHiveMetastore(thriftClient), IDENTITY), + new BridgingHiveMetastore(createThriftHiveMetastore(thriftClient)), listeningDecorator(newCachedThreadPool(daemonThreadsNamed("test-%s"))), new Duration(5, TimeUnit.MINUTES), Optional.of(new Duration(1, TimeUnit.MINUTES)), @@ -833,7 +830,7 @@ void omitsCacheForNumberOfOperations(int expectedCacheOmittingOperations) private CachingHiveMetastore createMetastoreWithDirectExecutor(CachingHiveMetastoreConfig config) { return cachingHiveMetastore( - new BridgingHiveMetastore(createThriftHiveMetastore(), IDENTITY), + new BridgingHiveMetastore(createThriftHiveMetastore()), directExecutor(), config.getMetastoreCacheTtl(), config.getMetastoreRefreshInterval(), diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestHiveGlueMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestHiveGlueMetastore.java index c1f6bc05bcaf..3025e3f47df5 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestHiveGlueMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestHiveGlueMetastore.java @@ -35,7 +35,6 @@ import io.trino.plugin.hive.HiveMetastoreClosure; import io.trino.plugin.hive.HiveType; import io.trino.plugin.hive.PartitionStatistics; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.HiveColumnStatistics; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.PartitionWithStatistics; @@ -208,7 +207,7 @@ public void setup() } @Override - protected HiveMetastore createMetastore(File tempDir, HiveIdentity identity) + protected HiveMetastore createMetastore(File tempDir) { GlueHiveMetastoreConfig glueConfig = new GlueHiveMetastoreConfig(); glueConfig.setDefaultWarehouseDir(tempDir.toURI().toString()); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java index e74003aae259..1dd1b844d976 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java @@ -20,7 +20,6 @@ import io.trino.plugin.hive.SchemaAlreadyExistsException; import io.trino.plugin.hive.TableAlreadyExistsException; import io.trino.plugin.hive.acid.AcidTransaction; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.HivePrincipal; import io.trino.plugin.hive.metastore.HivePrivilegeInfo; import io.trino.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege; @@ -106,7 +105,7 @@ public InMemoryThriftMetastore(File baseDirectory, ThriftMetastoreConfig metasto } @Override - public synchronized void createDatabase(HiveIdentity identity, Database database) + public synchronized void createDatabase(Database database) { requireNonNull(database, "database is null"); @@ -132,19 +131,19 @@ public synchronized void createDatabase(HiveIdentity identity, Database database // TODO: respect deleteData @Override - public synchronized void dropDatabase(HiveIdentity identity, String databaseName, boolean deleteData) + public synchronized void dropDatabase(String databaseName, boolean deleteData) { if (!databases.containsKey(databaseName)) { throw new SchemaNotFoundException(databaseName); } - if (!getAllTables(identity, databaseName).isEmpty()) { + if (!getAllTables(databaseName).isEmpty()) { throw new TrinoException(SCHEMA_NOT_EMPTY, "Schema not empty: " + databaseName); } databases.remove(databaseName); } @Override - public synchronized void alterDatabase(HiveIdentity identity, String databaseName, Database newDatabase) + public synchronized void alterDatabase(String databaseName, Database newDatabase) { String newDatabaseName = newDatabase.getName(); @@ -171,13 +170,13 @@ public synchronized void alterDatabase(HiveIdentity identity, String databaseNam } @Override - public synchronized List getAllDatabases(HiveIdentity identity) + public synchronized List getAllDatabases() { return ImmutableList.copyOf(databases.keySet()); } @Override - public synchronized void createTable(HiveIdentity identity, Table table) + public synchronized void createTable(Table table) { TableType tableType = TableType.valueOf(table.getTableType()); checkArgument(EnumSet.of(MANAGED_TABLE, EXTERNAL_TABLE, VIRTUAL_VIEW).contains(tableType), "Invalid table type: %s", tableType); @@ -211,9 +210,9 @@ public synchronized void createTable(HiveIdentity identity, Table table) } @Override - public synchronized void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) + public synchronized void dropTable(String databaseName, String tableName, boolean deleteData) { - List locations = listAllDataPaths(identity, this, databaseName, tableName); + List locations = listAllDataPaths(this, databaseName, tableName); SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); Table table = relations.remove(schemaTableName); @@ -235,10 +234,10 @@ public synchronized void dropTable(HiveIdentity identity, String databaseName, S } } - private static List listAllDataPaths(HiveIdentity identity, ThriftMetastore metastore, String schemaName, String tableName) + private static List listAllDataPaths(ThriftMetastore metastore, String schemaName, String tableName) { ImmutableList.Builder locations = builder(); - Table table = metastore.getTable(identity, schemaName, tableName).get(); + Table table = metastore.getTable(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 @@ -248,9 +247,9 @@ private static List listAllDataPaths(HiveIdentity identity, ThriftMetast List partitionColumnNames = table.getPartitionKeys().stream() .map(FieldSchema::getName) .collect(toImmutableList()); - Optional> partitionNames = metastore.getPartitionNamesByFilter(identity, schemaName, tableName, partitionColumnNames, TupleDomain.all()); + Optional> partitionNames = metastore.getPartitionNamesByFilter(schemaName, tableName, partitionColumnNames, TupleDomain.all()); if (partitionNames.isPresent()) { - metastore.getPartitionsByNames(identity, schemaName, tableName, partitionNames.get()).stream() + metastore.getPartitionsByNames(schemaName, tableName, partitionNames.get()).stream() .map(partition -> partition.getSd().getLocation()) .filter(location -> !location.startsWith(table.getSd().getLocation())) .forEach(locations::add); @@ -260,7 +259,7 @@ private static List listAllDataPaths(HiveIdentity identity, ThriftMetast } @Override - public synchronized void alterTable(HiveIdentity identity, String databaseName, String tableName, Table newTable) + public synchronized void alterTable(String databaseName, String tableName, Table newTable) { SchemaTableName oldName = new SchemaTableName(databaseName, tableName); SchemaTableName newName = new SchemaTableName(newTable.getDbName(), newTable.getTableName()); @@ -286,13 +285,13 @@ public synchronized void alterTable(HiveIdentity identity, String databaseName, } @Override - public void alterTransactionalTable(HiveIdentity identity, Table table, long transactionId, long writeId) + public void alterTransactionalTable(Table table, long transactionId, long writeId) { - alterTable(identity, table.getDbName(), table.getTableName(), table); + alterTable(table.getDbName(), table.getTableName(), table); } @Override - public synchronized List getAllTables(HiveIdentity identity, String databaseName) + public synchronized List getAllTables(String databaseName) { ImmutableList.Builder tables = ImmutableList.builder(); for (SchemaTableName schemaTableName : this.relations.keySet()) { @@ -304,7 +303,7 @@ public synchronized List getAllTables(HiveIdentity identity, String data } @Override - public synchronized List getTablesWithParameter(HiveIdentity identity, String databaseName, String parameterKey, String parameterValue) + public synchronized List getTablesWithParameter(String databaseName, String parameterKey, String parameterValue) { requireNonNull(parameterKey, "parameterKey is null"); requireNonNull(parameterValue, "parameterValue is null"); @@ -317,7 +316,7 @@ public synchronized List getTablesWithParameter(HiveIdentity identity, S } @Override - public synchronized List getAllViews(HiveIdentity identity, String databaseName) + public synchronized List getAllViews(String databaseName) { ImmutableList.Builder tables = ImmutableList.builder(); for (SchemaTableName schemaTableName : this.views.keySet()) { @@ -329,13 +328,13 @@ public synchronized List getAllViews(HiveIdentity identity, String datab } @Override - public synchronized Optional getDatabase(HiveIdentity identity, String databaseName) + public synchronized Optional getDatabase(String databaseName) { return Optional.ofNullable(databases.get(databaseName)); } @Override - public synchronized void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitionsWithStatistics) + public synchronized void addPartitions(String databaseName, String tableName, List partitionsWithStatistics) { for (PartitionWithStatistics partitionWithStatistics : partitionsWithStatistics) { Partition partition = toMetastoreApiPartition(partitionWithStatistics.getPartition()); @@ -349,14 +348,14 @@ public synchronized void addPartitions(HiveIdentity identity, String databaseNam } @Override - public synchronized void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData) + public synchronized void dropPartition(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(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partitionWithStatistics) + public synchronized void alterPartition(String databaseName, String tableName, PartitionWithStatistics partitionWithStatistics) { Partition partition = toMetastoreApiPartition(partitionWithStatistics.getPartition()); if (partition.getParameters() == null) { @@ -368,7 +367,7 @@ public synchronized void alterPartition(HiveIdentity identity, String databaseNa } @Override - public synchronized Optional getPartition(HiveIdentity identity, String databaseName, String tableName, List partitionValues) + public synchronized Optional getPartition(String databaseName, String tableName, List partitionValues) { PartitionName name = PartitionName.partition(databaseName, tableName, partitionValues); Partition partition = partitions.get(name); @@ -379,7 +378,7 @@ public synchronized Optional getPartition(HiveIdentity identity, Stri } @Override - public synchronized Optional> getPartitionNamesByFilter(HiveIdentity identity, String databaseName, String tableName, List columnNames, TupleDomain partitionKeysFilter) + public synchronized Optional> getPartitionNamesByFilter(String databaseName, String tableName, List columnNames, TupleDomain partitionKeysFilter) { Optional> parts = partitionKeyFilterToStringList(columnNames, partitionKeysFilter, assumeCanonicalPartitionKeys); @@ -412,7 +411,7 @@ private static boolean partitionMatches(Partition partition, String databaseName } @Override - public synchronized List getPartitionsByNames(HiveIdentity identity, String databaseName, String tableName, List partitionNames) + public synchronized List getPartitionsByNames(String databaseName, String tableName, List partitionNames) { ImmutableList.Builder builder = builder(); for (String name : partitionNames) { @@ -427,7 +426,7 @@ public synchronized List getPartitionsByNames(HiveIdentity identity, } @Override - public synchronized Optional
getTable(HiveIdentity identity, String databaseName, String tableName) + public synchronized Optional
getTable(String databaseName, String tableName) { SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); return Optional.ofNullable(relations.get(schemaTableName)); @@ -440,7 +439,7 @@ public Set getSupportedColumnStatistics(Type type) } @Override - public synchronized PartitionStatistics getTableStatistics(HiveIdentity identity, Table table) + public synchronized PartitionStatistics getTableStatistics(Table table) { return getTableStatistics(table.getDbName(), table.getTableName()); } @@ -456,7 +455,7 @@ private synchronized PartitionStatistics getTableStatistics(String databaseName, } @Override - public synchronized Map getPartitionStatistics(HiveIdentity identity, Table table, List partitions) + public synchronized Map getPartitionStatistics(Table table, List partitions) { List partitionColumns = table.getPartitionKeys().stream() .map(FieldSchema::getName) @@ -482,74 +481,74 @@ private synchronized Map getPartitionStatistics(Str } @Override - public synchronized void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, AcidTransaction transaction, Function update) + public synchronized void updateTableStatistics(String databaseName, String tableName, AcidTransaction transaction, Function update) { columnStatistics.put(new SchemaTableName(databaseName, tableName), update.apply(getTableStatistics(databaseName, tableName))); } @Override - public synchronized void updatePartitionStatistics(HiveIdentity identity, Table table, String partitionName, Function update) + public synchronized void updatePartitionStatistics(Table table, String partitionName, Function update) { PartitionName partitionKey = PartitionName.partition(table.getDbName(), table.getTableName(), partitionName); partitionColumnStatistics.put(partitionKey, update.apply(getPartitionStatistics(table.getDbName(), table.getTableName(), ImmutableSet.of(partitionName)).get(partitionName))); } @Override - public void createRole(HiveIdentity identity, String role, String grantor) + public void createRole(String role, String grantor) { throw new UnsupportedOperationException(); } @Override - public void dropRole(HiveIdentity identity, String role) + public void dropRole(String role) { throw new UnsupportedOperationException(); } @Override - public Set listRoles(HiveIdentity identity) + public Set listRoles() { throw new UnsupportedOperationException(); } @Override - public void grantRoles(HiveIdentity identity, Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) + public void grantRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) { throw new UnsupportedOperationException(); } @Override - public void revokeRoles(HiveIdentity identity, Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) + public void revokeRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) { throw new UnsupportedOperationException(); } @Override - public Set listGrantedPrincipals(HiveIdentity identity, String role) + public Set listGrantedPrincipals(String role) { throw new UnsupportedOperationException(); } @Override - public Set listRoleGrants(HiveIdentity identity, HivePrincipal principal) + public Set listRoleGrants(HivePrincipal principal) { throw new UnsupportedOperationException(); } @Override - public Set listTablePrivileges(HiveIdentity identity, String databaseName, String tableName, Optional tableOwner, Optional principal) + public Set listTablePrivileges(String databaseName, String tableName, Optional tableOwner, Optional principal) { return ImmutableSet.of(); } @Override - public void grantTablePrivileges(HiveIdentity identity, String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) + public void grantTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) { throw new UnsupportedOperationException(); } @Override - public void revokeTablePrivileges(HiveIdentity identity, String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) + public void revokeTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) { throw new UnsupportedOperationException(); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/S3HiveQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/S3HiveQueryRunner.java index ec0b87325a4e..d463d7e6d85a 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/S3HiveQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/S3HiveQueryRunner.java @@ -15,7 +15,6 @@ import com.google.common.net.HostAndPort; import io.trino.plugin.hive.HiveQueryRunner; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.containers.HiveMinioDataLake; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; import io.trino.testing.DistributedQueryRunner; @@ -133,8 +132,7 @@ public DistributedQueryRunner build() setMetastore(distributedQueryRunner -> new BridgingHiveMetastore( testingThriftHiveMetastoreBuilder() .metastoreClient(hiveMetastoreEndpoint) - .build(), - new HiveIdentity(distributedQueryRunner.getDefaultSession().getIdentity().toConnectorIdentity()))); + .build())); setInitialSchemasLocationBase("s3a://" + bucketName); // cannot use s3:// as Hive metastore is not configured to accept it return super.build(); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperations.java index 7dfa6b523beb..c5ba96df15af 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperations.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.iceberg.catalog.hms; -import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.AcidTransactionOwner; import io.trino.plugin.hive.metastore.MetastoreUtil; import io.trino.plugin.hive.metastore.PrincipalPrivileges; @@ -61,16 +60,14 @@ public HiveMetastoreTableOperations( protected void commitToExistingTable(TableMetadata base, TableMetadata metadata) { String newMetadataLocation = writeNewMetadata(metadata, version + 1); - HiveIdentity identity = new HiveIdentity(session.getIdentity()); long lockId = thriftMetastore.acquireTableExclusiveLock( - identity, new AcidTransactionOwner(session.getUser()), session.getQueryId(), database, tableName); try { - Table currentTable = fromMetastoreApiTable(thriftMetastore.getTable(identity, database, tableName) + Table currentTable = fromMetastoreApiTable(thriftMetastore.getTable(database, tableName) .orElseThrow(() -> new TableNotFoundException(getSchemaTableName()))); checkState(currentMetadataLocation != null, "No current metadata location for existing table"); @@ -98,7 +95,7 @@ protected void commitToExistingTable(TableMetadata base, TableMetadata metadata) } } finally { - thriftMetastore.releaseTableLock(identity, lockId); + thriftMetastore.releaseTableLock(lockId); } shouldRefresh = true; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperationsProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperationsProvider.java index 1b56413d4cc9..8ee2b606914b 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperationsProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperationsProvider.java @@ -52,7 +52,7 @@ public IcebergTableOperations createTableOperations( return new HiveMetastoreTableOperations( fileIoProvider.createFileIo(new HdfsContext(session), session.getQueryId()), ((TrinoHiveCatalog) catalog).getMetastore(), - thriftMetastoreFactory.createMetastore(), + thriftMetastoreFactory.createMetastore(Optional.of(session.getIdentity())), session, database, table,