From f4a2844a8bfafdb0fc5f5bfc0e6e723300c466f4 Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Sat, 2 Oct 2021 23:40:19 -0700 Subject: [PATCH] Remove per user metadata caching from CachingHiveMetastore --- .../plugin/hive/HiveMetadataFactory.java | 3 +- .../plugin/hive/HiveMetastoreClosure.java | 5 - .../plugin/hive/HivePageSinkProvider.java | 2 +- .../metastore/ForwardingHiveMetastore.java | 6 - .../plugin/hive/metastore/HiveMetastore.java | 2 - .../alluxio/AlluxioHiveMetastore.java | 6 - .../metastore/cache/CachingHiveMetastore.java | 298 +++++++----------- .../cache/SharedHiveMetastoreCache.java | 6 +- .../metastore/file/FileHiveMetastore.java | 6 - .../metastore/glue/GlueHiveMetastore.java | 6 - .../recording/RecordingHiveMetastore.java | 6 - .../thrift/BridgingHiveMetastore.java | 6 - .../trino/plugin/hive/AbstractTestHive.java | 1 + .../metastore/UnimplementedHiveMetastore.java | 6 - .../cache/TestCachingHiveMetastore.java | 10 +- .../plugin/iceberg/TrinoCatalogFactory.java | 3 +- .../CountingAccessFileHiveMetastore.java | 7 - 17 files changed, 122 insertions(+), 257 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java index 9419a743d084..a66404d04a3b 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java @@ -17,6 +17,7 @@ import io.airlift.json.JsonCodec; import io.airlift.units.Duration; import io.trino.plugin.base.CatalogName; +import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.MetastoreConfig; import io.trino.plugin.hive.metastore.SemiTransactionalHiveMetastore; @@ -192,7 +193,7 @@ public HiveMetadataFactory( public TransactionalMetadata create(ConnectorIdentity identity, boolean autoCommit) { HiveMetastoreClosure hiveMetastoreClosure = new HiveMetastoreClosure( - memoizeMetastore(metastoreFactory.createMetastore(Optional.of(identity)), perTransactionCacheMaximumSize)); // per-transaction cache + memoizeMetastore(metastoreFactory.createMetastore(Optional.of(identity)), new HiveIdentity(identity), perTransactionCacheMaximumSize)); // per-transaction cache SemiTransactionalHiveMetastore metastore = new SemiTransactionalHiveMetastore( hdfsEnvironment, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetastoreClosure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetastoreClosure.java index 433061926ba2..1091873b7c67 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetastoreClosure.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetastoreClosure.java @@ -301,11 +301,6 @@ public Set listTablePrivileges(String databaseName, String ta return delegate.listTablePrivileges(databaseName, tableName, tableOwner, principal); } - public boolean isImpersonationEnabled() - { - return delegate.isImpersonationEnabled(); - } - public long openTransaction(HiveIdentity identity) { return delegate.openTransaction(identity); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java index 6a4f8a6a80a9..624b0bf035ec 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java @@ -157,7 +157,7 @@ private ConnectorPageSink createPageSink(HiveWritableTableHandle handle, boolean session.getQueryId(), new HivePageSinkMetadataProvider( handle.getPageSinkMetadata(), - new HiveMetastoreClosure(memoizeMetastore(metastoreFactory.createMetastore(Optional.of(session.getIdentity())), perTransactionMetastoreCacheMaximumSize)), + new HiveMetastoreClosure(memoizeMetastore(metastoreFactory.createMetastore(Optional.of(session.getIdentity())), new HiveIdentity(session), perTransactionMetastoreCacheMaximumSize)), new HiveIdentity(session)), typeManager, hdfsEnvironment, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/ForwardingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/ForwardingHiveMetastore.java index e955f1b76022..3024335bc075 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/ForwardingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/ForwardingHiveMetastore.java @@ -356,12 +356,6 @@ public Set listTablePrivileges(String databaseName, return delegate.listTablePrivileges(databaseName, tableName, tableOwner, principal); } - @Override - public boolean isImpersonationEnabled() - { - return delegate.isImpersonationEnabled(); - } - @Override public long openTransaction(HiveIdentity identity) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastore.java index 4b87b0ca0ddb..b05415c1ed48 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastore.java @@ -143,8 +143,6 @@ default void updatePartitionStatistics(HiveIdentity identity, Table table, Strin */ Set listTablePrivileges(String databaseName, String tableName, Optional tableOwner, Optional principal); - boolean isImpersonationEnabled(); - default long openTransaction(HiveIdentity identity) { throw new UnsupportedOperationException(); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/alluxio/AlluxioHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/alluxio/AlluxioHiveMetastore.java index b324ebf031f4..952d56402ce5 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/alluxio/AlluxioHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/alluxio/AlluxioHiveMetastore.java @@ -479,10 +479,4 @@ public Set listTablePrivileges(String databaseName, String ta { throw new TrinoException(NOT_SUPPORTED, "listTablePrivileges"); } - - @Override - public boolean isImpersonationEnabled() - { - return false; - } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java index 50313456c12f..3a44bd9849e4 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.hive.metastore.cache; +import com.google.common.annotations.VisibleForTesting; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.collect.ImmutableList; @@ -61,7 +62,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Objects; import java.util.Optional; import java.util.OptionalLong; import java.util.Set; @@ -71,7 +71,6 @@ import java.util.function.Predicate; import static com.google.common.base.Functions.identity; -import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Throwables.throwIfInstanceOf; import static com.google.common.base.Throwables.throwIfUnchecked; @@ -104,26 +103,28 @@ public enum StatsRecording } protected final HiveMetastore delegate; + private final HiveIdentity identity; private final LoadingCache> databaseCache; private final LoadingCache> databaseNamesCache; - private final LoadingCache, Optional> tableCache; + private final LoadingCache> tableCache; private final LoadingCache> tableNamesCache; private final LoadingCache> tablesWithParameterCache; - private final LoadingCache, PartitionStatistics> tableStatisticsCache; - private final LoadingCache, PartitionStatistics> partitionStatisticsCache; + private final LoadingCache tableStatisticsCache; + private final LoadingCache partitionStatisticsCache; private final LoadingCache> viewNamesCache; - private final LoadingCache, Optional> partitionCache; - private final LoadingCache, Optional>> partitionFilterCache; + private final LoadingCache> partitionCache; + private final LoadingCache>> partitionFilterCache; private final LoadingCache> tablePrivilegesCache; private final LoadingCache> rolesCache; private final LoadingCache> roleGrantsCache; private final LoadingCache> grantedPrincipalsCache; private final LoadingCache> configValuesCache; - public static CachingHiveMetastore cachingHiveMetastore(HiveMetastore delegate, Executor executor, Duration cacheTtl, Optional refreshInterval, long maximumSize) + public static CachingHiveMetastore cachingHiveMetastore(HiveMetastore delegate, HiveIdentity identity, Executor executor, Duration cacheTtl, Optional refreshInterval, long maximumSize) { return new CachingHiveMetastore( delegate, + identity, OptionalLong.of(cacheTtl.toMillis()), refreshInterval .map(Duration::toMillis) @@ -134,10 +135,11 @@ public static CachingHiveMetastore cachingHiveMetastore(HiveMetastore delegate, StatsRecording.ENABLED); } - public static CachingHiveMetastore memoizeMetastore(HiveMetastore delegate, long maximumSize) + public static CachingHiveMetastore memoizeMetastore(HiveMetastore delegate, HiveIdentity identity, long maximumSize) { return new CachingHiveMetastore( delegate, + identity, OptionalLong.empty(), OptionalLong.empty(), Optional.empty(), @@ -145,9 +147,10 @@ public static CachingHiveMetastore memoizeMetastore(HiveMetastore delegate, long StatsRecording.DISABLED); } - protected CachingHiveMetastore(HiveMetastore delegate, OptionalLong expiresAfterWriteMillis, OptionalLong refreshMills, Optional executor, long maximumSize, StatsRecording statsRecording) + protected CachingHiveMetastore(HiveMetastore delegate, HiveIdentity identity, OptionalLong expiresAfterWriteMillis, OptionalLong refreshMills, Optional executor, long maximumSize, StatsRecording statsRecording) { this.delegate = requireNonNull(delegate, "delegate is null"); + this.identity = requireNonNull(identity, "identity is null"); requireNonNull(executor, "executor is null"); databaseNamesCache = buildCache(expiresAfterWriteMillis, refreshMills, executor, maximumSize, statsRecording, ignored -> loadAllDatabases()); @@ -256,17 +259,16 @@ private List loadAllDatabases() return delegate.getAllDatabases(); } - private Table getExistingTable(HiveIdentity identity, String databaseName, String tableName) + private Table getExistingTable(String databaseName, String tableName) { return getTable(identity, databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); } @Override - public Optional
getTable(HiveIdentity identity, String databaseName, String tableName) + public Optional
getTable(HiveIdentity ignored, String databaseName, String tableName) { - identity = updateIdentity(identity); - return get(tableCache, new WithIdentity<>(identity, hiveTableName(databaseName, tableName))); + return get(tableCache, hiveTableName(databaseName, tableName)); } @Override @@ -275,64 +277,63 @@ public Set getSupportedColumnStatistics(Type type) return delegate.getSupportedColumnStatistics(type); } - private Optional
loadTable(WithIdentity hiveTableName) + private Optional
loadTable(HiveTableName hiveTableName) { - return delegate.getTable(hiveTableName.getIdentity(), hiveTableName.key.getDatabaseName(), hiveTableName.key.getTableName()); + return delegate.getTable(identity, hiveTableName.getDatabaseName(), hiveTableName.getTableName()); } @Override - public PartitionStatistics getTableStatistics(HiveIdentity identity, Table table) + public PartitionStatistics getTableStatistics(HiveIdentity ignored, Table table) { - return get(tableStatisticsCache, new WithIdentity<>(updateIdentity(identity), hiveTableName(table.getDatabaseName(), table.getTableName()))); + return get(tableStatisticsCache, hiveTableName(table.getDatabaseName(), table.getTableName())); } - private PartitionStatistics loadTableColumnStatistics(WithIdentity hiveTableName) + private PartitionStatistics loadTableColumnStatistics(HiveTableName tableName) { - HiveTableName tableName = hiveTableName.getKey(); - Table table = getExistingTable(hiveTableName.getIdentity(), tableName.getDatabaseName(), tableName.getTableName()); - return delegate.getTableStatistics(hiveTableName.getIdentity(), table); + Table table = getExistingTable(tableName.getDatabaseName(), tableName.getTableName()); + return delegate.getTableStatistics(identity, table); } @Override - public Map getPartitionStatistics(HiveIdentity identity, Table table, List partitions) + public Map getPartitionStatistics(HiveIdentity ignored, Table table, List partitions) { HiveTableName hiveTableName = hiveTableName(table.getDatabaseName(), table.getTableName()); - List> partitionNames = partitions.stream() - .map(partition -> new WithIdentity<>(updateIdentity(identity), hivePartitionName(hiveTableName, makePartitionName(table, partition)))) + List partitionNames = partitions.stream() + .map(partition -> hivePartitionName(hiveTableName, makePartitionName(table, partition))) .collect(toImmutableList()); - Map, PartitionStatistics> statistics = getAll(partitionStatisticsCache, partitionNames); + Map statistics = getAll(partitionStatisticsCache, partitionNames); return statistics.entrySet() .stream() - .collect(toImmutableMap(entry -> entry.getKey().getKey().getPartitionName().get(), Entry::getValue)); + .collect(toImmutableMap(entry -> entry.getKey().getPartitionName().orElseThrow(), Entry::getValue)); } - private PartitionStatistics loadPartitionColumnStatistics(WithIdentity partition) + private PartitionStatistics loadPartitionColumnStatistics(HivePartitionName partition) { - HiveTableName tableName = partition.getKey().getHiveTableName(); - String partitionName = partition.getKey().getPartitionName().get(); - Table table = getExistingTable(partition.getIdentity(), tableName.getDatabaseName(), tableName.getTableName()); + HiveTableName tableName = partition.getHiveTableName(); + String partitionName = partition.getPartitionName().orElseThrow(); + Table table = getExistingTable(tableName.getDatabaseName(), tableName.getTableName()); Map partitionStatistics = delegate.getPartitionStatistics( - partition.getIdentity(), + identity, table, - ImmutableList.of(getExistingPartition(partition.getIdentity(), table, partition.getKey().getPartitionValues()))); + ImmutableList.of(getExistingPartition(table, partition.getPartitionValues()))); return partitionStatistics.get(partitionName); } - private Map, PartitionStatistics> loadPartitionsColumnStatistics(Iterable> keys) + private Map loadPartitionsColumnStatistics(Iterable keys) { - SetMultimap, WithIdentity> tablePartitions = stream(keys) - .collect(toImmutableSetMultimap(value -> new WithIdentity<>(value.getIdentity(), value.getKey().getHiveTableName()), Function.identity())); - ImmutableMap.Builder, PartitionStatistics> result = ImmutableMap.builder(); + SetMultimap tablePartitions = stream(keys) + .collect(toImmutableSetMultimap(HivePartitionName::getHiveTableName, Function.identity())); + ImmutableMap.Builder result = ImmutableMap.builder(); tablePartitions.keySet().forEach(tableName -> { - Set> partitionNames = tablePartitions.get(tableName); + Set partitionNames = tablePartitions.get(tableName); Set partitionNameStrings = partitionNames.stream() - .map(partitionName -> partitionName.getKey().getPartitionName().get()) + .map(partitionName -> partitionName.getPartitionName().orElseThrow()) .collect(toImmutableSet()); - Table table = getExistingTable(tableName.getIdentity(), tableName.getKey().getDatabaseName(), tableName.getKey().getTableName()); - List partitions = getExistingPartitionsByNames(tableName.getIdentity(), table, ImmutableList.copyOf(partitionNameStrings)); - Map statisticsByPartitionName = delegate.getPartitionStatistics(tableName.getIdentity(), table, partitions); - for (WithIdentity partitionName : partitionNames) { - String stringNameForPartition = partitionName.getKey().getPartitionName().get(); + Table table = getExistingTable(tableName.getDatabaseName(), tableName.getTableName()); + List partitions = getExistingPartitionsByNames(table, ImmutableList.copyOf(partitionNameStrings)); + Map statisticsByPartitionName = delegate.getPartitionStatistics(identity, table, partitions); + for (HivePartitionName partitionName : partitionNames) { + String stringNameForPartition = partitionName.getPartitionName().orElseThrow(); result.put(partitionName, statisticsByPartitionName.get(stringNameForPartition)); } }); @@ -340,48 +341,50 @@ private Map, PartitionStatistics> loadPartitions } @Override - public void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, AcidTransaction transaction, Function update) + public void updateTableStatistics( + HiveIdentity ignored, + String databaseName, + String tableName, + AcidTransaction transaction, + Function update) { - identity = updateIdentity(identity); try { delegate.updateTableStatistics(identity, databaseName, tableName, transaction, update); } finally { HiveTableName hiveTableName = hiveTableName(databaseName, tableName); - tableStatisticsCache.invalidate(new WithIdentity<>(identity, hiveTableName)); + tableStatisticsCache.invalidate(hiveTableName); // basic stats are stored as table properties - tableCache.invalidate(new WithIdentity<>(identity, hiveTableName)); + tableCache.invalidate(hiveTableName); } } @Override - public void updatePartitionStatistics(HiveIdentity identity, Table table, String partitionName, Function update) + public void updatePartitionStatistics(HiveIdentity ignored, Table table, String partitionName, Function update) { - identity = updateIdentity(identity); try { delegate.updatePartitionStatistics(identity, table, partitionName, update); } finally { HivePartitionName hivePartitionName = hivePartitionName(hiveTableName(table.getDatabaseName(), table.getTableName()), partitionName); - partitionStatisticsCache.invalidate(new WithIdentity<>(identity, hivePartitionName)); + partitionStatisticsCache.invalidate(hivePartitionName); // basic stats are stored as partition properties - partitionCache.invalidate(new WithIdentity<>(identity, hivePartitionName)); + partitionCache.invalidate(hivePartitionName); } } @Override - public void updatePartitionStatistics(HiveIdentity identity, Table table, Map> updates) + public void updatePartitionStatistics(HiveIdentity ignored, Table table, Map> updates) { try { - delegate.updatePartitionStatistics(updateIdentity(identity), table, updates); + delegate.updatePartitionStatistics(identity, table, updates); } finally { - HiveIdentity hiveIdentity = updateIdentity(identity); updates.forEach((partitionName, update) -> { HivePartitionName hivePartitionName = hivePartitionName(hiveTableName(table.getDatabaseName(), table.getTableName()), partitionName); - partitionStatisticsCache.invalidate(new WithIdentity<>(hiveIdentity, hivePartitionName)); + partitionStatisticsCache.invalidate(hivePartitionName); // basic stats are stored as partition properties - partitionCache.invalidate(new WithIdentity<>(hiveIdentity, hivePartitionName)); + partitionCache.invalidate(hivePartitionName); }); } } @@ -421,9 +424,8 @@ private List loadAllViews(String databaseName) } @Override - public void createDatabase(HiveIdentity identity, Database database) + public void createDatabase(HiveIdentity ignored, Database database) { - identity = updateIdentity(identity); try { delegate.createDatabase(identity, database); } @@ -433,9 +435,8 @@ public void createDatabase(HiveIdentity identity, Database database) } @Override - public void dropDatabase(HiveIdentity identity, String databaseName, boolean deleteData) + public void dropDatabase(HiveIdentity ignored, String databaseName, boolean deleteData) { - identity = updateIdentity(identity); try { delegate.dropDatabase(identity, databaseName, deleteData); } @@ -445,9 +446,8 @@ public void dropDatabase(HiveIdentity identity, String databaseName, boolean del } @Override - public void renameDatabase(HiveIdentity identity, String databaseName, String newDatabaseName) + public void renameDatabase(HiveIdentity ignored, String databaseName, String newDatabaseName) { - identity = updateIdentity(identity); try { delegate.renameDatabase(identity, databaseName, newDatabaseName); } @@ -458,9 +458,8 @@ public void renameDatabase(HiveIdentity identity, String databaseName, String ne } @Override - public void setDatabaseOwner(HiveIdentity identity, String databaseName, HivePrincipal principal) + public void setDatabaseOwner(HiveIdentity ignored, String databaseName, HivePrincipal principal) { - identity = updateIdentity(identity); try { delegate.setDatabaseOwner(identity, databaseName, principal); } @@ -476,9 +475,8 @@ protected void invalidateDatabase(String databaseName) } @Override - public void createTable(HiveIdentity identity, Table table, PrincipalPrivileges principalPrivileges) + public void createTable(HiveIdentity ignored, Table table, PrincipalPrivileges principalPrivileges) { - identity = updateIdentity(identity); try { delegate.createTable(identity, table, principalPrivileges); } @@ -488,9 +486,8 @@ public void createTable(HiveIdentity identity, Table table, PrincipalPrivileges } @Override - public void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) + public void dropTable(HiveIdentity ignored, String databaseName, String tableName, boolean deleteData) { - identity = updateIdentity(identity); try { delegate.dropTable(identity, databaseName, tableName, deleteData); } @@ -500,9 +497,8 @@ public void dropTable(HiveIdentity identity, String databaseName, String tableNa } @Override - public void replaceTable(HiveIdentity identity, String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) + public void replaceTable(HiveIdentity ignored, String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) { - identity = updateIdentity(identity); try { delegate.replaceTable(identity, databaseName, tableName, newTable, principalPrivileges); } @@ -513,9 +509,8 @@ public void replaceTable(HiveIdentity identity, String databaseName, String tabl } @Override - public void renameTable(HiveIdentity identity, String databaseName, String tableName, String newDatabaseName, String newTableName) + public void renameTable(HiveIdentity ignored, String databaseName, String tableName, String newDatabaseName, String newTableName) { - identity = updateIdentity(identity); try { delegate.renameTable(identity, databaseName, tableName, newDatabaseName, newTableName); } @@ -526,9 +521,8 @@ public void renameTable(HiveIdentity identity, String databaseName, String table } @Override - public void commentTable(HiveIdentity identity, String databaseName, String tableName, Optional comment) + public void commentTable(HiveIdentity ignored, String databaseName, String tableName, Optional comment) { - identity = updateIdentity(identity); try { delegate.commentTable(identity, databaseName, tableName, comment); } @@ -540,7 +534,6 @@ public void commentTable(HiveIdentity identity, String databaseName, String tabl @Override public void setTableOwner(HiveIdentity identity, String databaseName, String tableName, HivePrincipal principal) { - identity = updateIdentity(identity); try { delegate.setTableOwner(identity, databaseName, tableName, principal); } @@ -552,7 +545,6 @@ public void setTableOwner(HiveIdentity identity, String databaseName, String tab @Override public void commentColumn(HiveIdentity identity, String databaseName, String tableName, String columnName, Optional comment) { - identity = updateIdentity(identity); try { delegate.commentColumn(identity, databaseName, tableName, columnName, comment); } @@ -564,7 +556,6 @@ public void commentColumn(HiveIdentity identity, String databaseName, String tab @Override public void addColumn(HiveIdentity identity, String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) { - identity = updateIdentity(identity); try { delegate.addColumn(identity, databaseName, tableName, columnName, columnType, columnComment); } @@ -576,7 +567,6 @@ public void addColumn(HiveIdentity identity, String databaseName, String tableNa @Override public void renameColumn(HiveIdentity identity, String databaseName, String tableName, String oldColumnName, String newColumnName) { - identity = updateIdentity(identity); try { delegate.renameColumn(identity, databaseName, tableName, oldColumnName, newColumnName); } @@ -588,7 +578,6 @@ public void renameColumn(HiveIdentity identity, String databaseName, String tabl @Override public void dropColumn(HiveIdentity identity, String databaseName, String tableName, String columnName) { - identity = updateIdentity(identity); try { delegate.dropColumn(identity, databaseName, tableName, columnName); } @@ -597,7 +586,8 @@ public void dropColumn(HiveIdentity identity, String databaseName, String tableN } } - protected void invalidateTable(String databaseName, String tableName) + @VisibleForTesting + void invalidateTable(String databaseName, String tableName) { invalidateTableCache(databaseName, tableName); tableNamesCache.invalidate(databaseName); @@ -612,24 +602,24 @@ protected void invalidateTable(String databaseName, String tableName) private void invalidateTableCache(String databaseName, String tableName) { tableCache.asMap().keySet().stream() - .filter(table -> table.getKey().getDatabaseName().equals(databaseName) && table.getKey().getTableName().equals(tableName)) + .filter(table -> table.getDatabaseName().equals(databaseName) && table.getTableName().equals(tableName)) .forEach(tableCache::invalidate); } private void invalidateTableStatisticsCache(String databaseName, String tableName) { tableStatisticsCache.asMap().keySet().stream() - .filter(table -> table.getKey().getDatabaseName().equals(databaseName) && table.getKey().getTableName().equals(tableName)) + .filter(table -> table.getDatabaseName().equals(databaseName) && table.getTableName().equals(tableName)) .forEach(tableCache::invalidate); } - private Partition getExistingPartition(HiveIdentity identity, Table table, List partitionValues) + private Partition getExistingPartition(Table table, List partitionValues) { return getPartition(identity, table, partitionValues) .orElseThrow(() -> new PartitionNotFoundException(table.getSchemaTableName(), partitionValues)); } - private List getExistingPartitionsByNames(HiveIdentity identity, Table table, List partitionNames) + private List getExistingPartitionsByNames(Table table, List partitionNames) { Map partitions = getPartitionsByNames(identity, table, partitionNames).entrySet().stream() .map(entry -> immutableEntry(entry.getKey(), entry.getValue().orElseThrow(() -> @@ -644,56 +634,55 @@ private List getExistingPartitionsByNames(HiveIdentity identity, Tabl @Override public Optional getPartition(HiveIdentity identity, Table table, List partitionValues) { - return get(partitionCache, new WithIdentity<>(updateIdentity(identity), hivePartitionName(hiveTableName(table.getDatabaseName(), table.getTableName()), partitionValues))); + return get(partitionCache, hivePartitionName(hiveTableName(table.getDatabaseName(), table.getTableName()), partitionValues)); } @Override public Optional> getPartitionNamesByFilter(HiveIdentity identity, String databaseName, String tableName, List columnNames, TupleDomain partitionKeysFilter) { - return get(partitionFilterCache, new WithIdentity<>(updateIdentity(identity), partitionFilter(databaseName, tableName, columnNames, partitionKeysFilter))); + return get(partitionFilterCache, partitionFilter(databaseName, tableName, columnNames, partitionKeysFilter)); } - private Optional> loadPartitionNamesByFilter(WithIdentity partitionFilter) + private Optional> loadPartitionNamesByFilter(PartitionFilter partitionFilter) { return delegate.getPartitionNamesByFilter( - partitionFilter.getIdentity(), - partitionFilter.getKey().getHiveTableName().getDatabaseName(), - partitionFilter.getKey().getHiveTableName().getTableName(), - partitionFilter.getKey().getPartitionColumnNames(), - partitionFilter.getKey().getPartitionKeysFilter()); + identity, + partitionFilter.getHiveTableName().getDatabaseName(), + partitionFilter.getHiveTableName().getTableName(), + partitionFilter.getPartitionColumnNames(), + partitionFilter.getPartitionKeysFilter()); } @Override public Map> getPartitionsByNames(HiveIdentity identity, Table table, List partitionNames) { - List> names = partitionNames.stream() - .map(name -> new WithIdentity<>(updateIdentity(identity), hivePartitionName(hiveTableName(table.getDatabaseName(), table.getTableName()), name))) + List names = partitionNames.stream() + .map(name -> hivePartitionName(hiveTableName(table.getDatabaseName(), table.getTableName()), name)) .collect(toImmutableList()); - Map, Optional> all = getAll(partitionCache, names); + Map> all = getAll(partitionCache, names); ImmutableMap.Builder> partitionsByName = ImmutableMap.builder(); - for (Entry, Optional> entry : all.entrySet()) { - partitionsByName.put(entry.getKey().getKey().getPartitionName().get(), entry.getValue()); + for (Entry> entry : all.entrySet()) { + partitionsByName.put(entry.getKey().getPartitionName().orElseThrow(), entry.getValue()); } return partitionsByName.buildOrThrow(); } - private Optional loadPartitionByName(WithIdentity partitionName) + private Optional loadPartitionByName(HivePartitionName partitionName) { - HiveTableName hiveTableName = partitionName.getKey().getHiveTableName(); - return getTable(partitionName.getIdentity(), hiveTableName.getDatabaseName(), hiveTableName.getTableName()) - .flatMap(table -> delegate.getPartition(partitionName.getIdentity(), table, partitionName.getKey().getPartitionValues())); + HiveTableName hiveTableName = partitionName.getHiveTableName(); + return getTable(identity, hiveTableName.getDatabaseName(), hiveTableName.getTableName()) + .flatMap(table -> delegate.getPartition(identity, table, partitionName.getPartitionValues())); } - private Map, Optional> loadPartitionsByNames(Iterable> partitionNames) + private Map> loadPartitionsByNames(Iterable partitionNames) { requireNonNull(partitionNames, "partitionNames is null"); checkArgument(!Iterables.isEmpty(partitionNames), "partitionNames is empty"); - WithIdentity firstPartition = Iterables.get(partitionNames, 0); + HivePartitionName firstPartition = Iterables.get(partitionNames, 0); - HiveTableName hiveTableName = firstPartition.getKey().getHiveTableName(); - HiveIdentity identity = updateIdentity(firstPartition.getIdentity()); + HiveTableName hiveTableName = firstPartition.getHiveTableName(); Optional
table = getTable(identity, hiveTableName.getDatabaseName(), hiveTableName.getTableName()); if (table.isEmpty()) { return stream(partitionNames) @@ -701,16 +690,15 @@ private Map, Optional> loadPartitions } List partitionsToFetch = new ArrayList<>(); - for (WithIdentity partitionName : partitionNames) { - checkArgument(partitionName.getKey().getHiveTableName().equals(hiveTableName), "Expected table name %s but got %s", hiveTableName, partitionName.getKey().getHiveTableName()); - checkArgument(identity.equals(partitionName.getIdentity()), "Expected identity %s but got %s", identity, partitionName.getIdentity()); - partitionsToFetch.add(partitionName.getKey().getPartitionName().get()); + for (HivePartitionName partitionName : partitionNames) { + checkArgument(partitionName.getHiveTableName().equals(hiveTableName), "Expected table name %s but got %s", hiveTableName, partitionName.getHiveTableName()); + partitionsToFetch.add(partitionName.getPartitionName().orElseThrow()); } - ImmutableMap.Builder, Optional> partitions = ImmutableMap.builder(); + ImmutableMap.Builder> partitions = ImmutableMap.builder(); Map> partitionsByNames = delegate.getPartitionsByNames(identity, table.get(), partitionsToFetch); - for (WithIdentity partitionName : partitionNames) { - partitions.put(partitionName, partitionsByNames.getOrDefault(partitionName.getKey().getPartitionName().get(), Optional.empty())); + for (HivePartitionName partitionName : partitionNames) { + partitions.put(partitionName, partitionsByNames.getOrDefault(partitionName.getPartitionName().orElseThrow(), Optional.empty())); } return partitions.buildOrThrow(); } @@ -718,7 +706,6 @@ private Map, Optional> loadPartitions @Override public void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitions) { - identity = updateIdentity(identity); try { delegate.addPartitions(identity, databaseName, tableName, partitions); } @@ -731,7 +718,6 @@ public void addPartitions(HiveIdentity identity, String databaseName, String tab @Override public void dropPartition(HiveIdentity identity, String databaseName, String tableName, List parts, boolean deleteData) { - identity = updateIdentity(identity); try { delegate.dropPartition(identity, databaseName, tableName, parts, deleteData); } @@ -743,7 +729,6 @@ public void dropPartition(HiveIdentity identity, String databaseName, String tab @Override public void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partition) { - identity = updateIdentity(identity); try { delegate.alterPartition(identity, databaseName, tableName, partition); } @@ -839,15 +824,14 @@ private void invalidatePartitionCache(String databaseName, String tableName, Pre { HiveTableName hiveTableName = hiveTableName(databaseName, tableName); - Predicate> hivePartitionPredicate = - partitionName -> partitionName.getKey().getHiveTableName().equals(hiveTableName) && - partitionPredicate.test(partitionName.getKey().getPartitionName()); + Predicate hivePartitionPredicate = partitionName -> partitionName.getHiveTableName().equals(hiveTableName) && + partitionPredicate.test(partitionName.getPartitionName()); partitionCache.asMap().keySet().stream() .filter(hivePartitionPredicate) .forEach(partitionCache::invalidate); partitionFilterCache.asMap().keySet().stream() - .filter(partitionFilter -> partitionFilter.getKey().getHiveTableName().equals(hiveTableName)) + .filter(partitionFilter -> partitionFilter.getHiveTableName().equals(hiveTableName)) .forEach(partitionFilterCache::invalidate); partitionStatisticsCache.asMap().keySet().stream() .filter(hivePartitionPredicate) @@ -878,7 +862,7 @@ public void revokeTablePrivileges(String databaseName, String tableName, String private void invalidateTablePrivilegeCacheEntries(String databaseName, String tableName, String tableOwner, HivePrincipal grantee) { - // some callers of xxxxTablePrivileges use Optional.of(grantee), some Optional.empty() (to get all privileges), so have to invalidate them both + // some callers of table privilege methods use Optional.of(grantee), some Optional.empty() (to get all privileges), so have to invalidate them both tablePrivilegesCache.invalidate(new UserTableKey(Optional.of(grantee), databaseName, tableName, Optional.of(tableOwner))); tablePrivilegesCache.invalidate(new UserTableKey(Optional.empty(), databaseName, tableName, Optional.of(tableOwner))); } @@ -936,12 +920,6 @@ public String getValidWriteIds(HiveIdentity identity, List tabl return delegate.getValidWriteIds(identity, tables, currentTransactionId); } - @Override - public boolean isImpersonationEnabled() - { - return delegate.isImpersonationEnabled(); - } - private Set loadTablePrivileges(String databaseName, String tableName, Optional tableOwner, Optional principal) { return delegate.listTablePrivileges(databaseName, tableName, tableOwner, principal); @@ -973,7 +951,6 @@ public void updateTableWriteId(HiveIdentity identity, String dbName, String tabl @Override public void alterPartitions(HiveIdentity identity, String dbName, String tableName, List partitions, long writeId) { - identity = updateIdentity(identity); try { delegate.alterPartitions(identity, dbName, tableName, partitions, writeId); } @@ -985,7 +962,6 @@ 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) { - identity = updateIdentity(identity); try { delegate.addDynamicPartitions(identity, dbName, tableName, partitionNames, transactionId, writeId, operation); } @@ -1011,9 +987,9 @@ private static LoadingCache buildCache( Optional refreshExecutor, long maximumSize, StatsRecording statsRecording, - com.google.common.base.Function loader) + Function loader) { - CacheLoader cacheLoader = CacheLoader.from(loader); + CacheLoader cacheLoader = CacheLoader.from(loader::apply); checkArgument(refreshMillis.isEmpty() || refreshExecutor.isPresent(), "refreshMillis is provided but refreshExecutor is not"); if (refreshMillis.isPresent() && (expiresAfterWriteMillis.isEmpty() || expiresAfterWriteMillis.getAsLong() > refreshMillis.getAsLong())) { @@ -1064,66 +1040,6 @@ public Map loadAll(Iterable keys) cacheLoader); } - private static class WithIdentity - { - private final HiveIdentity identity; - private final T key; - private final int hashCode; - - public WithIdentity(HiveIdentity identity, T key) - { - this.identity = requireNonNull(identity, "identity is null"); - this.key = requireNonNull(key, "key is null"); - this.hashCode = Objects.hash(identity, key); - } - - public HiveIdentity getIdentity() - { - return identity; - } - - public T getKey() - { - return key; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - WithIdentity other = (WithIdentity) o; - return hashCode == other.hashCode && - Objects.equals(identity, other.identity) && - Objects.equals(key, other.key); - } - - @Override - public int hashCode() - { - return hashCode; - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("identity", identity) - .add("key", key) - .toString(); - } - } - - private HiveIdentity updateIdentity(HiveIdentity identity) - { - // remove identity if not doing impersonation - return delegate.isImpersonationEnabled() ? identity : HiveIdentity.none(); - } - @Managed @Nested public CacheStatsMBean getDatabaseStats() diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/SharedHiveMetastoreCache.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/SharedHiveMetastoreCache.java index f1f723156fb3..773a7e15535a 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/SharedHiveMetastoreCache.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/SharedHiveMetastoreCache.java @@ -19,6 +19,7 @@ import com.google.common.util.concurrent.UncheckedExecutionException; import io.airlift.units.Duration; import io.trino.plugin.base.CatalogName; +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.NodeManager; @@ -126,6 +127,7 @@ public HiveMetastoreFactory createCachingHiveMetastoreFactory(HiveMetastoreFacto // In case there are no empty executor slots, such operation would deadlock. Therefore, a reentrant executor needs to be // used. metastoreFactory.createMetastore(Optional.empty()), + HiveIdentity.none(), new ReentrantBoundedExecutor(executorService, maxMetastoreRefreshThreads), metastoreCacheTtl, metastoreRefreshInterval, @@ -200,8 +202,10 @@ public HiveMetastore createMetastore(Optional identity) private HiveMetastore createUserCachingMetastore(String user) { + ConnectorIdentity identity = ConnectorIdentity.ofUser(user); return cachingHiveMetastore( - metastoreFactory.createMetastore(Optional.of(ConnectorIdentity.ofUser(user))), + metastoreFactory.createMetastore(Optional.of(identity)), + new HiveIdentity(identity), new ReentrantBoundedExecutor(executorService, maxMetastoreRefreshThreads), metastoreCacheTtl, metastoreRefreshInterval, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java index 01b659da7e60..ff6c413e031e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java @@ -1146,12 +1146,6 @@ public synchronized void revokeTablePrivileges(String databaseName, String table setTablePrivileges(grantee, databaseName, tableName, Sets.difference(currentPrivileges, privilegesToRemove)); } - @Override - public boolean isImpersonationEnabled() - { - return false; - } - private synchronized void setTablePrivileges( HivePrincipal grantee, String databaseName, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java index e5975af58264..64e12a83be5c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java @@ -1128,10 +1128,4 @@ public Set listTablePrivileges(String databaseName, String ta { return ImmutableSet.of(); } - - @Override - public boolean isImpersonationEnabled() - { - return false; - } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/recording/RecordingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/recording/RecordingHiveMetastore.java index 1a1915e45bed..9270c89300dc 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/recording/RecordingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/recording/RecordingHiveMetastore.java @@ -356,12 +356,6 @@ public Set listRoleGrants(HivePrincipal principal) () -> delegate.listRoleGrants(principal)); } - @Override - public boolean isImpersonationEnabled() - { - return delegate.isImpersonationEnabled(); - } - private void verifyRecordingMode() { if (recording.isReplay()) { 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 901a4f4ed3ae..9fb86d7d0e7b 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 @@ -463,12 +463,6 @@ public Set listTablePrivileges(String databaseName, String ta return delegate.listTablePrivileges(databaseName, tableName, tableOwner, principal); } - @Override - public boolean isImpersonationEnabled() - { - return delegate.isImpersonationEnabled(); - } - @Override public Optional getConfigValue(String name) { 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 e58b74a6f030..c327e4022c2e 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 @@ -791,6 +791,7 @@ protected final void setup(String host, int port, String databaseName, String ti new ThriftMetastoreConfig(), hdfsEnvironment, false)), + HiveIdentity.none(), executor, new Duration(1, MINUTES), Optional.of(new Duration(15, SECONDS)), diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/UnimplementedHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/UnimplementedHiveMetastore.java index d2e17253559c..09f998c86a42 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/UnimplementedHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/UnimplementedHiveMetastore.java @@ -277,10 +277,4 @@ public Set listRoleGrants(HivePrincipal principal) { throw new UnsupportedOperationException(); } - - @Override - public boolean isImpersonationEnabled() - { - throw new UnsupportedOperationException(); - } } 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 9ba23646e0b4..e6f870140be2 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 @@ -125,6 +125,7 @@ public void setUp() executor = listeningDecorator(newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s"))); metastore = cachingHiveMetastore( new BridgingHiveMetastore(thriftHiveMetastore), + IDENTITY, executor, new Duration(5, TimeUnit.MINUTES), Optional.of(new Duration(1, TimeUnit.MINUTES)), @@ -480,6 +481,7 @@ public void testCachingHiveMetastoreCreationViaMemoize() ThriftHiveMetastore thriftHiveMetastore = createThriftHiveMetastore(); metastore = (CachingHiveMetastore) memoizeMetastore( new BridgingHiveMetastore(thriftHiveMetastore), + IDENTITY, 1000); assertEquals(mockClient.getAccessCount(), 0); @@ -566,17 +568,12 @@ public Map> getPartitionsByNames(HiveIdentity identi return result; } - - @Override - public boolean isImpersonationEnabled() - { - return false; - } }; // Caching metastore metastore = cachingHiveMetastore( mockMetastore, + IDENTITY, executor, new Duration(5, TimeUnit.MINUTES), Optional.of(new Duration(1, TimeUnit.MINUTES)), @@ -670,6 +667,7 @@ private CachingHiveMetastore createMetastoreWithDirectExecutor(CachingHiveMetast { return (CachingHiveMetastore) cachingHiveMetastore( new BridgingHiveMetastore(createThriftHiveMetastore()), + IDENTITY, directExecutor(), config.getMetastoreCacheTtl(), config.getMetastoreRefreshInterval(), diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TrinoCatalogFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TrinoCatalogFactory.java index a189c7d68d23..4a9f40d1a1c4 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TrinoCatalogFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TrinoCatalogFactory.java @@ -17,6 +17,7 @@ import io.trino.plugin.hive.HdfsEnvironment; import io.trino.plugin.hive.HiveConfig; import io.trino.plugin.hive.NodeVersion; +import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.spi.TrinoException; @@ -77,7 +78,7 @@ public TrinoCatalog create(ConnectorIdentity identity) case HIVE_METASTORE: return new TrinoHiveCatalog( catalogName, - memoizeMetastore(metastoreFactory.createMetastore(Optional.of(identity)), 1000), + memoizeMetastore(metastoreFactory.createMetastore(Optional.of(identity)), new HiveIdentity(identity), 1000), hdfsEnvironment, typeManager, tableOperationsProvider, diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/CountingAccessFileHiveMetastore.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/CountingAccessFileHiveMetastore.java index be279f7c388c..c144f28ef3c5 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/CountingAccessFileHiveMetastore.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/CountingAccessFileHiveMetastore.java @@ -299,13 +299,6 @@ public Set listTablePrivileges(String databaseName, String ta throw new UnsupportedOperationException(); } - @Override - public boolean isImpersonationEnabled() - { - // Local operation, doesn't need to be included in methodInvocations - return delegate.isImpersonationEnabled(); - } - @Override public PartitionStatistics getTableStatistics(HiveIdentity identity, Table table) {