Skip to content

Commit

Permalink
[Enhancement](multi-catalog) Make meta cache batch loading concurrent…
Browse files Browse the repository at this point in the history
…ly. (apache#21471)

I will enhance performance about querying meta cache of hms tables by 2 steps:
**Step1** : use concurrent batch loading for meta cache
**Step2** : execute some other tasks concurrently as soon as possible

**This pr mainly for step1 and it mainly do the following things:**
- Create a `CacheBulkLoader` for batch loading
- Remove the executor of the previous async cache loader and change the loader's type to `CacheBulkLoader` (We do not set any refresh strategies for LoadingCache, so the previous executor is not useful)
- Use a `FixedCacheThreadPool` to replace the `CacheThreadPool` (The previous `CacheThreadPool` just log warn infos and will not throw any exceptions when the pool is full).
- Remove parallel streams and use the `CacheBulkLoader` to do batch loadings
- Change the value of `max_external_cache_loader_thread_pool_size` to 64, and set the pool size of hms client pool to `max_external_cache_loader_thread_pool_size`
- Fix the spelling mistake for `max_hive_table_catch_num`
  • Loading branch information
dutyu authored Jul 6, 2023
1 parent fde73b6 commit bb3b677
Show file tree
Hide file tree
Showing 8 changed files with 232 additions and 100 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1694,7 +1694,7 @@ public class Config extends ConfigBase {

@ConfField(mutable = false, masterOnly = false, description = {"Hive表到分区名列表缓存的最大数量。",
"Max cache number of hive table to partition names list."})
public static long max_hive_table_catch_num = 1000;
public static long max_hive_table_cache_num = 1000;

@ConfField(mutable = false, masterOnly = false, description = {"获取Hive分区值时候的最大返回数量,-1代表没有限制。",
"Max number of hive partition values to return while list partitions, -1 means no limitation."})
Expand All @@ -1705,7 +1705,7 @@ public class Config extends ConfigBase {
* Max thread pool size for loading external meta cache
*/
@ConfField(mutable = false, masterOnly = false)
public static int max_external_cache_loader_thread_pool_size = 10;
public static int max_external_cache_loader_thread_pool_size = 64;

/**
* Max cache num of external catalog's file
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,14 @@ public static ThreadPoolExecutor newDaemonFixedThreadPool(int numThread,
poolName, needRegisterMetric);
}

public static ThreadPoolExecutor newDaemonFixedThreadPool(int numThread, int queueSize,
String poolName, int timeoutSeconds,
boolean needRegisterMetric) {
return newDaemonThreadPool(numThread, numThread, KEEP_ALIVE_TIME, TimeUnit.SECONDS,
new LinkedBlockingQueue<>(queueSize), new BlockedPolicy(poolName, timeoutSeconds),
poolName, needRegisterMetric);
}

public static <T> ThreadPoolExecutor newDaemonFixedPriorityThreadPool(int numThread, int initQueueSize,
Comparator<T> comparator, Class<T> tClass,
String poolName, boolean needRegisterMetric) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you 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 org.apache.doris.common.util;

import org.apache.doris.common.Pair;

import com.google.common.cache.CacheLoader;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.common.collect.Streams;

import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.stream.Collectors;

public abstract class CacheBulkLoader<K, V> extends CacheLoader<K, V> {

protected abstract ExecutorService getExecutor();

@Override
public Map<K, V> loadAll(Iterable<? extends K> keys)
throws ExecutionException, InterruptedException {
List<Pair<? extends K, Future<V>>> pList = Streams.stream(keys)
.map(key -> Pair.of(key, getExecutor().submit(() -> load(key))))
.collect(Collectors.toList());

Map<K, V> vMap = Maps.newLinkedHashMap();
for (Pair<? extends K, Future<V>> p : pList) {
vMap.put(p.first, p.second.get());
}
return ImmutableMap.copyOf(vMap);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@

import java.util.List;
import java.util.Map;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;

/**
* Cache meta of external catalog
Expand All @@ -44,11 +44,13 @@ public class ExternalMetaCacheMgr {
private Map<Long, HiveMetaStoreCache> cacheMap = Maps.newConcurrentMap();
// catalog id -> table schema cache
private Map<Long, ExternalSchemaCache> schemaCacheMap = Maps.newHashMap();
private Executor executor;
private ExecutorService executor;

public ExternalMetaCacheMgr() {
executor = ThreadPoolManager.newDaemonCacheThreadPool(Config.max_external_cache_loader_thread_pool_size,
"ExternalMetaCacheMgr", true);
executor = ThreadPoolManager.newDaemonFixedThreadPool(
Config.max_external_cache_loader_thread_pool_size,
Config.max_external_cache_loader_thread_pool_size * 1000,
"ExternalMetaCacheMgr", 120, true);
}

public HiveMetaStoreCache getMetaStoreCache(HMSExternalCatalog catalog) {
Expand All @@ -69,7 +71,7 @@ public ExternalSchemaCache getSchemaCache(ExternalCatalog catalog) {
if (cache == null) {
synchronized (schemaCacheMap) {
if (!schemaCacheMap.containsKey(catalog.getId())) {
schemaCacheMap.put(catalog.getId(), new ExternalSchemaCache(catalog, executor));
schemaCacheMap.put(catalog.getId(), new ExternalSchemaCache(catalog));
}
cache = schemaCacheMap.get(catalog.getId());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,31 +37,30 @@
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;

// The schema cache for external table
public class ExternalSchemaCache {
private static final Logger LOG = LogManager.getLogger(ExternalSchemaCache.class);
private ExternalCatalog catalog;
private final ExternalCatalog catalog;

private LoadingCache<SchemaCacheKey, ImmutableList<Column>> schemaCache;

public ExternalSchemaCache(ExternalCatalog catalog, Executor executor) {
public ExternalSchemaCache(ExternalCatalog catalog) {
this.catalog = catalog;
init(executor);
init();
initMetrics();
}

private void init(Executor executor) {
private void init() {
schemaCache = CacheBuilder.newBuilder().maximumSize(Config.max_external_schema_cache_num)
.expireAfterAccess(Config.external_cache_expire_time_minutes_after_access, TimeUnit.MINUTES)
.build(CacheLoader.asyncReloading(new CacheLoader<SchemaCacheKey, ImmutableList<Column>>() {
.build(new CacheLoader<SchemaCacheKey, ImmutableList<Column>>() {
@Override
public ImmutableList<Column> load(SchemaCacheKey key) throws Exception {
public ImmutableList<Column> load(SchemaCacheKey key) {
return loadSchema(key);
}
}, executor));
});
}

private void initMetrics() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@
public class HMSExternalCatalog extends ExternalCatalog {
private static final Logger LOG = LogManager.getLogger(HMSExternalCatalog.class);

private static final int MAX_CLIENT_POOL_SIZE = 8;
private static final int MIN_CLIENT_POOL_SIZE = 8;
protected PooledHiveMetaStoreClient client;
// Record the latest synced event id when processing hive events
// Must set to -1 otherwise client.getNextNotification will throw exception
Expand Down Expand Up @@ -161,7 +161,8 @@ protected void initLocalObjectsImpl() {
}
}

client = new PooledHiveMetaStoreClient(hiveConf, MAX_CLIENT_POOL_SIZE);
client = new PooledHiveMetaStoreClient(hiveConf,
Math.max(MIN_CLIENT_POOL_SIZE, Config.max_external_cache_loader_thread_pool_size));
}

@Override
Expand Down
Loading

0 comments on commit bb3b677

Please sign in to comment.