HBASE-25528: Dedicated merge dispatch threadpool on master
Adds "hbase.master.executor.merge.dispatch.threads" and defaults to 2. Also adds additional logging that includes the number of split plans and merge plans computed for each normalizer run.
This commit is contained in:
parent
b07549febb
commit
36b4698cea
|
@ -1605,6 +1605,13 @@ public final class HConstants {
|
|||
"hbase.master.executor.serverops.threads";
|
||||
public static final int MASTER_SERVER_OPERATIONS_THREADS_DEFAULT = 5;
|
||||
|
||||
/**
|
||||
* Number of threads used to dispatch merge operations to the regionservers.
|
||||
*/
|
||||
public static final String MASTER_MERGE_DISPATCH_THREADS =
|
||||
"hbase.master.executor.merge.dispatch.threads";
|
||||
public static final int MASTER_MERGE_DISPATCH_THREADS_DEFAULT = 2;
|
||||
|
||||
public static final String MASTER_META_SERVER_OPERATIONS_THREADS =
|
||||
"hbase.master.executor.meta.serverops.threads";
|
||||
public static final int MASTER_META_SERVER_OPERATIONS_THREADS_DEFAULT = 5;
|
||||
|
|
|
@ -152,7 +152,7 @@ public enum EventType {
|
|||
* C_M_MERGE_REGION<br>
|
||||
* Client asking Master to merge regions.
|
||||
*/
|
||||
C_M_MERGE_REGION (30, ExecutorType.MASTER_TABLE_OPERATIONS),
|
||||
C_M_MERGE_REGION (30, ExecutorType.MASTER_MERGE_OPERATIONS),
|
||||
/**
|
||||
* Messages originating from Client to Master.<br>
|
||||
* C_M_DELETE_TABLE<br>
|
||||
|
|
|
@ -35,6 +35,7 @@ public enum ExecutorType {
|
|||
MASTER_META_SERVER_OPERATIONS (6),
|
||||
M_LOG_REPLAY_OPS (7),
|
||||
MASTER_SNAPSHOT_OPERATIONS (8),
|
||||
MASTER_MERGE_OPERATIONS (9),
|
||||
|
||||
// RegionServer executor services
|
||||
RS_OPEN_REGION (20),
|
||||
|
|
|
@ -1324,6 +1324,9 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
HConstants.MASTER_LOG_REPLAY_OPS_THREADS, HConstants.MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT));
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS, conf.getInt(
|
||||
SnapshotManager.SNAPSHOT_POOL_THREADS_KEY, SnapshotManager.SNAPSHOT_POOL_THREADS_DEFAULT));
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_MERGE_OPERATIONS, conf.getInt(
|
||||
HConstants.MASTER_MERGE_DISPATCH_THREADS,
|
||||
HConstants.MASTER_MERGE_DISPATCH_THREADS_DEFAULT));
|
||||
|
||||
// We depend on there being only one instance of this executor running
|
||||
// at a time. To do concurrency, would need fencing of enable/disable of
|
||||
|
|
|
@ -209,14 +209,21 @@ class SimpleRegionNormalizer implements RegionNormalizer, ConfigurationObserver
|
|||
ctx.getTableRegions().size());
|
||||
|
||||
final List<NormalizationPlan> plans = new ArrayList<>();
|
||||
int splitPlansCount = 0;
|
||||
if (proceedWithSplitPlanning) {
|
||||
plans.addAll(computeSplitNormalizationPlans(ctx));
|
||||
List<NormalizationPlan> splitPlans = computeSplitNormalizationPlans(ctx);
|
||||
splitPlansCount = splitPlans.size();
|
||||
plans.addAll(splitPlans);
|
||||
}
|
||||
int mergePlansCount = 0;
|
||||
if (proceedWithMergePlanning) {
|
||||
plans.addAll(computeMergeNormalizationPlans(ctx));
|
||||
List<NormalizationPlan> mergePlans = computeMergeNormalizationPlans(ctx);
|
||||
mergePlansCount = mergePlans.size();
|
||||
plans.addAll(mergePlans);
|
||||
}
|
||||
|
||||
LOG.debug("Computed {} normalization plans for table {}", plans.size(), table);
|
||||
LOG.debug("Computed normalization plans for table {}. Total plans: {}, split plans: {}, " +
|
||||
"merge plans: {}", table, plans.size(), splitPlansCount, mergePlansCount);
|
||||
return plans;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue