Skip to content

Commit

Permalink
[Enhancement](multi-catalog) expose config: hms client pool size.
Browse files Browse the repository at this point in the history
  • Loading branch information
王翔宇 committed Jun 29, 2023
1 parent 16c218f commit 129df15
Show file tree
Hide file tree
Showing 3 changed files with 7 additions and 4 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 @@ -1707,6 +1707,10 @@ public class Config extends ConfigBase {
@ConfField(mutable = false, masterOnly = false)
public static int max_external_cache_loader_thread_pool_size = 10;

@ConfField(mutable = false, masterOnly = false, description = {"Hive Metastore 池实例数。",
"Max number of hive metastore client pool."})
public static short max_hms_client_pool_size = 8;

/**
* Max cache num of external catalog's file
* Decrease this value if FE's memory is small
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,6 @@
public class HMSExternalCatalog extends ExternalCatalog {
private static final Logger LOG = LogManager.getLogger(HMSExternalCatalog.class);

private static final int MAX_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 @@ -156,7 +155,7 @@ protected void initLocalObjectsImpl() {
}
}

client = new PooledHiveMetaStoreClient(hiveConf, MAX_CLIENT_POOL_SIZE);
client = new PooledHiveMetaStoreClient(hiveConf, Config.max_hms_client_pool_size);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,7 @@ public HiveMetaStoreCache(HMSExternalCatalog catalog, Executor executor) {
}

private void init() {
partitionValuesCache = CacheBuilder.newBuilder().maximumSize(Config.max_hive_table_catch_num)
partitionValuesCache = CacheBuilder.newBuilder().maximumSize(Config.max_hive_table_cache_num)
.expireAfterAccess(Config.external_cache_expire_time_minutes_after_access, TimeUnit.MINUTES)
.build(CacheLoader.asyncReloading(
new CacheLoader<PartitionValueCacheKey, HivePartitionValues>() {
Expand Down

0 comments on commit 129df15

Please sign in to comment.