All Downloads are FREE. Search and download functionalities are using the official Maven repository.

io.prestosql.plugin.hive.metastore.cache.CachingHiveMetastore Maven / Gradle / Ivy

There is a newer version: 350
Show newest version
/*
 * Licensed under the Apache License, Version 2.0 (the "License");
 * you may not use this file except in compliance with the License.
 * You may obtain a copy of the License at
 *
 *     http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */
package io.prestosql.plugin.hive.metastore.cache;

import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.SetMultimap;
import com.google.common.util.concurrent.UncheckedExecutionException;
import io.airlift.jmx.CacheStatsMBean;
import io.airlift.units.Duration;
import io.prestosql.plugin.hive.HivePartition;
import io.prestosql.plugin.hive.HiveType;
import io.prestosql.plugin.hive.PartitionNotFoundException;
import io.prestosql.plugin.hive.PartitionStatistics;
import io.prestosql.plugin.hive.authentication.HiveIdentity;
import io.prestosql.plugin.hive.metastore.Database;
import io.prestosql.plugin.hive.metastore.HiveMetastore;
import io.prestosql.plugin.hive.metastore.HivePartitionName;
import io.prestosql.plugin.hive.metastore.HivePrincipal;
import io.prestosql.plugin.hive.metastore.HivePrivilegeInfo;
import io.prestosql.plugin.hive.metastore.HiveTableName;
import io.prestosql.plugin.hive.metastore.Partition;
import io.prestosql.plugin.hive.metastore.PartitionFilter;
import io.prestosql.plugin.hive.metastore.PartitionWithStatistics;
import io.prestosql.plugin.hive.metastore.PrincipalPrivileges;
import io.prestosql.plugin.hive.metastore.Table;
import io.prestosql.plugin.hive.metastore.TablesWithParameterCacheKey;
import io.prestosql.plugin.hive.metastore.UserTableKey;
import io.prestosql.spi.PrestoException;
import io.prestosql.spi.connector.SchemaTableName;
import io.prestosql.spi.connector.TableNotFoundException;
import io.prestosql.spi.predicate.TupleDomain;
import io.prestosql.spi.security.RoleGrant;
import io.prestosql.spi.statistics.ColumnStatisticType;
import io.prestosql.spi.type.Type;
import org.weakref.jmx.Managed;
import org.weakref.jmx.Nested;

import javax.annotation.concurrent.ThreadSafe;

import java.util.ArrayList;
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;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.function.Function;

import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Throwables.throwIfInstanceOf;
import static com.google.common.base.Throwables.throwIfUnchecked;
import static com.google.common.cache.CacheLoader.asyncReloading;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.ImmutableMap.toImmutableMap;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static com.google.common.collect.ImmutableSetMultimap.toImmutableSetMultimap;
import static com.google.common.collect.Maps.immutableEntry;
import static com.google.common.collect.Streams.stream;
import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService;
import static io.prestosql.plugin.hive.HivePartitionManager.extractPartitionValues;
import static io.prestosql.plugin.hive.metastore.HivePartitionName.hivePartitionName;
import static io.prestosql.plugin.hive.metastore.HiveTableName.hiveTableName;
import static io.prestosql.plugin.hive.metastore.MetastoreUtil.makePartitionName;
import static io.prestosql.plugin.hive.metastore.PartitionFilter.partitionFilter;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;

/**
 * Hive Metastore Cache
 */
@ThreadSafe
public class CachingHiveMetastore
        implements HiveMetastore
{
    public enum StatsRecording
    {
        ENABLED,
        DISABLED
    }

    protected final HiveMetastore delegate;
    private final LoadingCache> databaseCache;
    private final LoadingCache> databaseNamesCache;
    private final LoadingCache, Optional> tableCache;
    private final LoadingCache> tableNamesCache;
    private final LoadingCache> tablesWithParameterCache;
    private final LoadingCache, PartitionStatistics> tableStatisticsCache;
    private final LoadingCache, PartitionStatistics> partitionStatisticsCache;
    private final LoadingCache> viewNamesCache;
    private final LoadingCache, Optional> partitionCache;
    private final LoadingCache, Optional>> partitionFilterCache;
    private final LoadingCache> tablePrivilegesCache;
    private final LoadingCache> rolesCache;
    private final LoadingCache> roleGrantsCache;
    private final LoadingCache> grantedPrincipalsCache;
    private final LoadingCache> configValuesCache;

    public static HiveMetastore cachingHiveMetastore(HiveMetastore delegate, Executor executor, CachingHiveMetastoreConfig config)
    {
        return cachingHiveMetastore(
                delegate,
                executor,
                config.getMetastoreCacheTtl(),
                config.getMetastoreRefreshInterval(),
                config.getMetastoreCacheMaximumSize());
    }

    public static HiveMetastore cachingHiveMetastore(HiveMetastore delegate, Executor executor, Duration cacheTtl, Optional refreshInterval, long maximumSize)
    {
        if (cacheTtl.toMillis() == 0 || maximumSize == 0) {
            // caching is disabled
            return delegate;
        }

        return new CachingHiveMetastore(
                delegate,
                executor,
                OptionalLong.of(cacheTtl.toMillis()),
                refreshInterval
                        .map(Duration::toMillis)
                        .map(OptionalLong::of)
                        .orElseGet(OptionalLong::empty),
                maximumSize,
                StatsRecording.ENABLED);
    }

    public static CachingHiveMetastore memoizeMetastore(HiveMetastore delegate, long maximumSize)
    {
        return new CachingHiveMetastore(
                delegate,
                newDirectExecutorService(),
                OptionalLong.empty(),
                OptionalLong.empty(),
                maximumSize,
                StatsRecording.DISABLED);
    }

    protected CachingHiveMetastore(HiveMetastore delegate, Executor executor, OptionalLong expiresAfterWriteMillis, OptionalLong refreshMills, long maximumSize, StatsRecording statsRecording)
    {
        this.delegate = requireNonNull(delegate, "delegate is null");
        requireNonNull(executor, "executor is null");

        databaseNamesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(this::loadAllDatabases), executor));

        databaseCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(this::loadDatabase), executor));

        tableNamesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(this::loadAllTables), executor));

        tablesWithParameterCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(this::loadTablesMatchingParameter), executor));

        tableStatisticsCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(this::loadTableColumnStatistics), executor));

        partitionStatisticsCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(new CacheLoader<>()
                {
                    @Override
                    public PartitionStatistics load(WithIdentity key)
                    {
                        return loadPartitionColumnStatistics(key);
                    }

                    @Override
                    public Map, PartitionStatistics> loadAll(Iterable> keys)
                    {
                        return loadPartitionColumnStatistics(keys);
                    }
                }, executor));

        tableCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(this::loadTable), executor));

        viewNamesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(this::loadAllViews), executor));

        partitionFilterCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(this::loadPartitionNamesByFilter), executor));

        partitionCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(new CacheLoader<>()
                {
                    @Override
                    public Optional load(WithIdentity partitionName)
                    {
                        return loadPartitionByName(partitionName);
                    }

                    @Override
                    public Map, Optional> loadAll(Iterable> partitionNames)
                    {
                        return loadPartitionsByNames(partitionNames);
                    }
                }, executor));

        tablePrivilegesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(key -> loadTablePrivileges(key.getDatabase(), key.getTable(), key.getOwner(), key.getPrincipal())), executor));

        rolesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(this::loadRoles), executor));

        roleGrantsCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(this::loadRoleGrants), executor));

        grantedPrincipalsCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(this::loadPrincipals), executor));

        configValuesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize, statsRecording)
                .build(asyncReloading(CacheLoader.from(this::loadConfigValue), executor));
    }

    @Managed
    public void flushCache()
    {
        databaseNamesCache.invalidateAll();
        tableNamesCache.invalidateAll();
        viewNamesCache.invalidateAll();
        databaseCache.invalidateAll();
        tableCache.invalidateAll();
        partitionCache.invalidateAll();
        partitionFilterCache.invalidateAll();
        tablePrivilegesCache.invalidateAll();
        tableStatisticsCache.invalidateAll();
        partitionStatisticsCache.invalidateAll();
        rolesCache.invalidateAll();
    }

    private static  V get(LoadingCache cache, K key)
    {
        try {
            return cache.getUnchecked(key);
        }
        catch (UncheckedExecutionException e) {
            throwIfInstanceOf(e.getCause(), PrestoException.class);
            throw e;
        }
    }

    private static  Map getAll(LoadingCache cache, Iterable keys)
    {
        try {
            return cache.getAll(keys);
        }
        catch (ExecutionException | UncheckedExecutionException e) {
            throwIfInstanceOf(e.getCause(), PrestoException.class);
            throwIfUnchecked(e);
            throw new UncheckedExecutionException(e);
        }
    }

    @Override
    public Optional getDatabase(String databaseName)
    {
        return get(databaseCache, databaseName);
    }

    private Optional loadDatabase(String databaseName)
    {
        return delegate.getDatabase(databaseName);
    }

    @Override
    public List getAllDatabases()
    {
        return get(databaseNamesCache, "");
    }

    private List loadAllDatabases()
    {
        return delegate.getAllDatabases();
    }

    private Table getExistingTable(HiveIdentity identity, 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) { identity = updateIdentity(identity); return get(tableCache, new WithIdentity<>(identity, hiveTableName(databaseName, tableName))); } @Override public Set getSupportedColumnStatistics(Type type) { return delegate.getSupportedColumnStatistics(type); } private Optional
loadTable(WithIdentity hiveTableName) { return delegate.getTable(hiveTableName.getIdentity(), hiveTableName.key.getDatabaseName(), hiveTableName.key.getTableName()); } @Override public PartitionStatistics getTableStatistics(HiveIdentity identity, Table table) { return get(tableStatisticsCache, new WithIdentity<>(updateIdentity(identity), hiveTableName(table.getDatabaseName(), table.getTableName()))); } private PartitionStatistics loadTableColumnStatistics(WithIdentity hiveTableName) { HiveTableName tableName = hiveTableName.getKey(); Table table = getExistingTable(hiveTableName.getIdentity(), tableName.getDatabaseName(), tableName.getTableName()); return delegate.getTableStatistics(hiveTableName.getIdentity(), table); } @Override public Map getPartitionStatistics(HiveIdentity identity, 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)))) .collect(toImmutableList()); Map, PartitionStatistics> statistics = getAll(partitionStatisticsCache, partitionNames); return statistics.entrySet() .stream() .collect(toImmutableMap(entry -> entry.getKey().getKey().getPartitionName().get(), Entry::getValue)); } private PartitionStatistics loadPartitionColumnStatistics(WithIdentity partition) { HiveTableName tableName = partition.getKey().getHiveTableName(); String partitionName = partition.getKey().getPartitionName().get(); Table table = getExistingTable(partition.getIdentity(), tableName.getDatabaseName(), tableName.getTableName()); Map partitionStatistics = delegate.getPartitionStatistics( partition.getIdentity(), table, ImmutableList.of(getExistingPartition(partition.getIdentity(), table, partition.getKey().getPartitionValues()))); return partitionStatistics.get(partitionName); } private Map, PartitionStatistics> loadPartitionColumnStatistics(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(); tablePartitions.keySet().forEach(tableName -> { Set> partitionNames = tablePartitions.get(tableName); Set partitionNameStrings = partitionNames.stream() .map(partitionName -> partitionName.getKey().getPartitionName().get()) .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(); result.put(partitionName, statisticsByPartitionName.get(stringNameForPartition)); } }); return result.build(); } @Override public void updateTableStatistics(HiveIdentity identity, String databaseName, String tableName, Function update) { identity = updateIdentity(identity); try { delegate.updateTableStatistics(identity, databaseName, tableName, update); } finally { HiveTableName hiveTableName = hiveTableName(databaseName, tableName); tableStatisticsCache.invalidate(new WithIdentity<>(identity, hiveTableName)); // basic stats are stored as table properties tableCache.invalidate(new WithIdentity<>(identity, hiveTableName)); } } @Override public void updatePartitionStatistics(HiveIdentity identity, 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)); // basic stats are stored as partition properties partitionCache.invalidate(new WithIdentity<>(identity, hivePartitionName)); } } @Override public List getAllTables(String databaseName) { return get(tableNamesCache, databaseName); } private List loadAllTables(String databaseName) { return delegate.getAllTables(databaseName); } @Override public List getTablesWithParameter(String databaseName, String parameterKey, String parameterValue) { TablesWithParameterCacheKey key = new TablesWithParameterCacheKey(databaseName, parameterKey, parameterValue); return get(tablesWithParameterCache, key); } private List loadTablesMatchingParameter(TablesWithParameterCacheKey key) { return delegate.getTablesWithParameter(key.getDatabaseName(), key.getParameterKey(), key.getParameterValue()); } @Override public List getAllViews(String databaseName) { return get(viewNamesCache, databaseName); } private List loadAllViews(String databaseName) { return delegate.getAllViews(databaseName); } @Override public void createDatabase(HiveIdentity identity, Database database) { identity = updateIdentity(identity); try { delegate.createDatabase(identity, database); } finally { invalidateDatabase(database.getDatabaseName()); } } @Override public void dropDatabase(HiveIdentity identity, String databaseName) { identity = updateIdentity(identity); try { delegate.dropDatabase(identity, databaseName); } finally { invalidateDatabase(databaseName); } } @Override public void renameDatabase(HiveIdentity identity, String databaseName, String newDatabaseName) { identity = updateIdentity(identity); try { delegate.renameDatabase(identity, databaseName, newDatabaseName); } finally { invalidateDatabase(databaseName); invalidateDatabase(newDatabaseName); } } @Override public void setDatabaseOwner(HiveIdentity identity, String databaseName, HivePrincipal principal) { identity = updateIdentity(identity); try { delegate.setDatabaseOwner(identity, databaseName, principal); } finally { invalidateDatabase(databaseName); } } protected void invalidateDatabase(String databaseName) { databaseCache.invalidate(databaseName); databaseNamesCache.invalidateAll(); } @Override public void createTable(HiveIdentity identity, Table table, PrincipalPrivileges principalPrivileges) { identity = updateIdentity(identity); try { delegate.createTable(identity, table, principalPrivileges); } finally { invalidateTable(table.getDatabaseName(), table.getTableName()); } } @Override public void dropTable(HiveIdentity identity, String databaseName, String tableName, boolean deleteData) { identity = updateIdentity(identity); try { delegate.dropTable(identity, databaseName, tableName, deleteData); } finally { invalidateTable(databaseName, tableName); } } @Override public void replaceTable(HiveIdentity identity, String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) { identity = updateIdentity(identity); try { delegate.replaceTable(identity, databaseName, tableName, newTable, principalPrivileges); } finally { invalidateTable(databaseName, tableName); invalidateTable(newTable.getDatabaseName(), newTable.getTableName()); } } @Override public void renameTable(HiveIdentity identity, String databaseName, String tableName, String newDatabaseName, String newTableName) { identity = updateIdentity(identity); try { delegate.renameTable(identity, databaseName, tableName, newDatabaseName, newTableName); } finally { invalidateTable(databaseName, tableName); invalidateTable(newDatabaseName, newTableName); } } @Override public void commentTable(HiveIdentity identity, String databaseName, String tableName, Optional comment) { identity = updateIdentity(identity); try { delegate.commentTable(identity, databaseName, tableName, comment); } finally { invalidateTable(databaseName, tableName); } } @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); } finally { invalidateTable(databaseName, tableName); } } @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); } finally { invalidateTable(databaseName, tableName); } } @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); } finally { invalidateTable(databaseName, tableName); } } @Override public void dropColumn(HiveIdentity identity, String databaseName, String tableName, String columnName) { identity = updateIdentity(identity); try { delegate.dropColumn(identity, databaseName, tableName, columnName); } finally { invalidateTable(databaseName, tableName); } } protected void invalidateTable(String databaseName, String tableName) { invalidateTableCache(databaseName, tableName); tableNamesCache.invalidate(databaseName); viewNamesCache.invalidate(databaseName); tablePrivilegesCache.asMap().keySet().stream() .filter(userTableKey -> userTableKey.matches(databaseName, tableName)) .forEach(tablePrivilegesCache::invalidate); invalidateTableStatisticsCache(databaseName, tableName); invalidatePartitionCache(databaseName, tableName); } private void invalidateTableCache(String databaseName, String tableName) { tableCache.asMap().keySet().stream() .filter(table -> table.getKey().getDatabaseName().equals(databaseName) && table.getKey().getTableName().equals(tableName)) .forEach(tableCache::invalidate); } private void invalidateTableStatisticsCache(String databaseName, String tableName) { tableStatisticsCache.asMap().keySet().stream() .filter(table -> table.getKey().getDatabaseName().equals(databaseName) && table.getKey().getTableName().equals(tableName)) .forEach(tableCache::invalidate); } private Partition getExistingPartition(HiveIdentity identity, Table table, List partitionValues) { return getPartition(identity, table, partitionValues) .orElseThrow(() -> new PartitionNotFoundException(table.getSchemaTableName(), partitionValues)); } private List getExistingPartitionsByNames(HiveIdentity identity, Table table, List partitionNames) { Map partitions = getPartitionsByNames(identity, table, partitionNames).entrySet().stream() .map(entry -> immutableEntry(entry.getKey(), entry.getValue().orElseThrow(() -> new PartitionNotFoundException(table.getSchemaTableName(), extractPartitionValues(entry.getKey()))))) .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue)); return partitionNames.stream() .map(partitions::get) .collect(toImmutableList()); } @Override public Optional getPartition(HiveIdentity identity, Table table, List partitionValues) { return get(partitionCache, new WithIdentity<>(updateIdentity(identity), hivePartitionName(hiveTableName(table.getDatabaseName(), table.getTableName()), partitionValues))); } @Override public Optional> getPartitionNamesByFilter(HiveIdentity identity, String databaseName, String tableName, List columnNames, TupleDomain partitionKeysFilter) { if (partitionKeysFilter.isNone()) { return Optional.of(ImmutableList.of()); } return get(partitionFilterCache, new WithIdentity<>(updateIdentity(identity), partitionFilter(databaseName, tableName, columnNames, partitionKeysFilter))); } private Optional> loadPartitionNamesByFilter(WithIdentity partitionFilter) { return delegate.getPartitionNamesByFilter( partitionFilter.getIdentity(), partitionFilter.getKey().getHiveTableName().getDatabaseName(), partitionFilter.getKey().getHiveTableName().getTableName(), partitionFilter.getKey().getPartitionColumnNames(), partitionFilter.getKey().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))) .collect(toImmutableList()); Map, Optional> all = getAll(partitionCache, names); ImmutableMap.Builder> partitionsByName = ImmutableMap.builder(); for (Entry, Optional> entry : all.entrySet()) { partitionsByName.put(entry.getKey().getKey().getPartitionName().get(), entry.getValue()); } return partitionsByName.build(); } private Optional loadPartitionByName(WithIdentity partitionName) { HiveTableName hiveTableName = partitionName.getKey().getHiveTableName(); return getTable(partitionName.getIdentity(), hiveTableName.getDatabaseName(), hiveTableName.getTableName()) .flatMap(table -> delegate.getPartition(partitionName.getIdentity(), table, partitionName.getKey().getPartitionValues())); } private Map, Optional> loadPartitionsByNames(Iterable> partitionNames) { requireNonNull(partitionNames, "partitionNames is null"); checkArgument(!Iterables.isEmpty(partitionNames), "partitionNames is empty"); WithIdentity firstPartition = Iterables.get(partitionNames, 0); HiveTableName hiveTableName = firstPartition.getKey().getHiveTableName(); HiveIdentity identity = updateIdentity(firstPartition.getIdentity()); Optional
table = getTable(identity, hiveTableName.getDatabaseName(), hiveTableName.getTableName()); if (table.isEmpty()) { return stream(partitionNames) .collect(toImmutableMap(name -> name, name -> Optional.empty())); } 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()); } ImmutableMap.Builder, Optional> 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())); } return partitions.build(); } @Override public void addPartitions(HiveIdentity identity, String databaseName, String tableName, List partitions) { identity = updateIdentity(identity); try { delegate.addPartitions(identity, databaseName, tableName, partitions); } finally { // todo do we need to invalidate all partitions? invalidatePartitionCache(databaseName, tableName); } } @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); } finally { invalidatePartitionCache(databaseName, tableName); } } @Override public void alterPartition(HiveIdentity identity, String databaseName, String tableName, PartitionWithStatistics partition) { identity = updateIdentity(identity); try { delegate.alterPartition(identity, databaseName, tableName, partition); } finally { invalidatePartitionCache(databaseName, tableName); } } @Override public void createRole(String role, String grantor) { try { delegate.createRole(role, grantor); } finally { rolesCache.invalidateAll(); } } @Override public void dropRole(String role) { try { delegate.dropRole(role); } finally { rolesCache.invalidateAll(); roleGrantsCache.invalidateAll(); } } @Override public Set listRoles() { return get(rolesCache, ""); } private Set loadRoles() { return delegate.listRoles(); } @Override public void grantRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) { try { delegate.grantRoles(roles, grantees, adminOption, grantor); } finally { roleGrantsCache.invalidateAll(); } } @Override public void revokeRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) { try { delegate.revokeRoles(roles, grantees, adminOption, grantor); } finally { roleGrantsCache.invalidateAll(); } } @Override public Set listGrantedPrincipals(String role) { return get(grantedPrincipalsCache, role); } @Override public Set listRoleGrants(HivePrincipal principal) { return get(roleGrantsCache, principal); } private Set loadRoleGrants(HivePrincipal principal) { return delegate.listRoleGrants(principal); } private Set loadPrincipals(String role) { return delegate.listGrantedPrincipals(role); } private void invalidatePartitionCache(String databaseName, String tableName) { HiveTableName hiveTableName = hiveTableName(databaseName, tableName); partitionCache.asMap().keySet().stream() .filter(partitionName -> partitionName.getKey().getHiveTableName().equals(hiveTableName)) .forEach(partitionCache::invalidate); partitionFilterCache.asMap().keySet().stream() .filter(partitionFilter -> partitionFilter.getKey().getHiveTableName().equals(hiveTableName)) .forEach(partitionFilterCache::invalidate); partitionStatisticsCache.asMap().keySet().stream() .filter(partitionFilter -> partitionFilter.getKey().getHiveTableName().equals(hiveTableName)) .forEach(partitionStatisticsCache::invalidate); } @Override public void grantTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, Set privileges) { try { delegate.grantTablePrivileges(databaseName, tableName, tableOwner, grantee, privileges); } finally { invalidateTablePrivilegeCacheEntries(databaseName, tableName, tableOwner, grantee); } } @Override public void revokeTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, Set privileges) { try { delegate.revokeTablePrivileges(databaseName, tableName, tableOwner, grantee, privileges); } finally { invalidateTablePrivilegeCacheEntries(databaseName, tableName, tableOwner, grantee); } } 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 tablePrivilegesCache.invalidate(new UserTableKey(Optional.of(grantee), databaseName, tableName, tableOwner)); tablePrivilegesCache.invalidate(new UserTableKey(Optional.empty(), databaseName, tableName, tableOwner)); } @Override public Set listTablePrivileges(String databaseName, String tableName, String tableOwner, Optional principal) { return get(tablePrivilegesCache, new UserTableKey(principal, databaseName, tableName, tableOwner)); } @Override public Optional getConfigValue(String name) { return get(configValuesCache, name); } private Optional loadConfigValue(String name) { return delegate.getConfigValue(name); } @Override public long openTransaction(HiveIdentity identity) { return delegate.openTransaction(identity); } @Override public void commitTransaction(HiveIdentity identity, long transactionId) { delegate.commitTransaction(identity, transactionId); } @Override public void sendTransactionHeartbeat(HiveIdentity identity, long transactionId) { delegate.sendTransactionHeartbeat(identity, transactionId); } @Override public void acquireSharedReadLock(HiveIdentity identity, String queryId, long transactionId, List fullTables, List partitions) { delegate.acquireSharedReadLock(identity, queryId, transactionId, fullTables, partitions); } @Override public String getValidWriteIds(HiveIdentity identity, List tables, long currentTransactionId) { return delegate.getValidWriteIds(identity, tables, currentTransactionId); } @Override public boolean isImpersonationEnabled() { return delegate.isImpersonationEnabled(); } private Set loadTablePrivileges(String databaseName, String tableName, String tableOwner, Optional principal) { return delegate.listTablePrivileges(databaseName, tableName, tableOwner, principal); } private static CacheBuilder newCacheBuilder(OptionalLong expiresAfterWriteMillis, OptionalLong refreshMillis, long maximumSize, StatsRecording statsRecording) { CacheBuilder cacheBuilder = CacheBuilder.newBuilder(); if (expiresAfterWriteMillis.isPresent()) { cacheBuilder = cacheBuilder.expireAfterWrite(expiresAfterWriteMillis.getAsLong(), MILLISECONDS); } if (refreshMillis.isPresent() && (expiresAfterWriteMillis.isEmpty() || expiresAfterWriteMillis.getAsLong() > refreshMillis.getAsLong())) { cacheBuilder = cacheBuilder.refreshAfterWrite(refreshMillis.getAsLong(), MILLISECONDS); } cacheBuilder = cacheBuilder.maximumSize(maximumSize); if (statsRecording == StatsRecording.ENABLED) { cacheBuilder = cacheBuilder.recordStats(); } return cacheBuilder; } 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() { return new CacheStatsMBean(databaseCache); } @Managed @Nested public CacheStatsMBean getDatabaseNamesStats() { return new CacheStatsMBean(databaseNamesCache); } @Managed @Nested public CacheStatsMBean getTableStats() { return new CacheStatsMBean(tableCache); } @Managed @Nested public CacheStatsMBean getTableNamesStats() { return new CacheStatsMBean(tableNamesCache); } @Managed @Nested public CacheStatsMBean getTableWithParameterStats() { return new CacheStatsMBean(tablesWithParameterCache); } @Managed @Nested public CacheStatsMBean getTableStatisticsStats() { return new CacheStatsMBean(tableStatisticsCache); } @Managed @Nested public CacheStatsMBean getPartitionStatisticsStats() { return new CacheStatsMBean(partitionStatisticsCache); } @Managed @Nested public CacheStatsMBean getViewNamesStats() { return new CacheStatsMBean(viewNamesCache); } @Managed @Nested public CacheStatsMBean getPartitionStats() { return new CacheStatsMBean(partitionCache); } @Managed @Nested public CacheStatsMBean getPartitionFilterStats() { return new CacheStatsMBean(partitionFilterCache); } @Managed @Nested public CacheStatsMBean getTablePrivilegesStats() { return new CacheStatsMBean(tablePrivilegesCache); } @Managed @Nested public CacheStatsMBean getRolesStats() { return new CacheStatsMBean(rolesCache); } @Managed @Nested public CacheStatsMBean getRoleGrantsStats() { return new CacheStatsMBean(roleGrantsCache); } @Managed @Nested public CacheStatsMBean getGrantedPrincipalsStats() { return new CacheStatsMBean(grantedPrincipalsCache); } @Managed @Nested public CacheStatsMBean getConfigValuesStats() { return new CacheStatsMBean(configValuesCache); } }