Skip to content

Commit de51a40

Browse files
authored
HBASE-25528: Dedicated merge dispatch threadpool on master (#2904)
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. Signed-off-by: Wellington Chevreuil <wchevreuil@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org>
1 parent 4aff481 commit de51a40

5 files changed

Lines changed: 22 additions & 4 deletions

File tree

hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1605,6 +1605,13 @@ public enum OperationStatusCode {
16051605
"hbase.master.executor.serverops.threads";
16061606
public static final int MASTER_SERVER_OPERATIONS_THREADS_DEFAULT = 5;
16071607

1608+
/**
1609+
* Number of threads used to dispatch merge operations to the regionservers.
1610+
*/
1611+
public static final String MASTER_MERGE_DISPATCH_THREADS =
1612+
"hbase.master.executor.merge.dispatch.threads";
1613+
public static final int MASTER_MERGE_DISPATCH_THREADS_DEFAULT = 2;
1614+
16081615
public static final String MASTER_META_SERVER_OPERATIONS_THREADS =
16091616
"hbase.master.executor.meta.serverops.threads";
16101617
public static final int MASTER_META_SERVER_OPERATIONS_THREADS_DEFAULT = 5;

hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -152,7 +152,7 @@ public enum EventType {
152152
* C_M_MERGE_REGION<br>
153153
* Client asking Master to merge regions.
154154
*/
155-
C_M_MERGE_REGION (30, ExecutorType.MASTER_TABLE_OPERATIONS),
155+
C_M_MERGE_REGION (30, ExecutorType.MASTER_MERGE_OPERATIONS),
156156
/**
157157
* Messages originating from Client to Master.<br>
158158
* C_M_DELETE_TABLE<br>

hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ public enum ExecutorType {
3535
MASTER_META_SERVER_OPERATIONS (6),
3636
M_LOG_REPLAY_OPS (7),
3737
MASTER_SNAPSHOT_OPERATIONS (8),
38+
MASTER_MERGE_OPERATIONS (9),
3839

3940
// RegionServer executor services
4041
RS_OPEN_REGION (20),

hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1324,6 +1324,9 @@ private void startServiceThreads() throws IOException {
13241324
HConstants.MASTER_LOG_REPLAY_OPS_THREADS, HConstants.MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT));
13251325
this.executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS, conf.getInt(
13261326
SnapshotManager.SNAPSHOT_POOL_THREADS_KEY, SnapshotManager.SNAPSHOT_POOL_THREADS_DEFAULT));
1327+
this.executorService.startExecutorService(ExecutorType.MASTER_MERGE_OPERATIONS, conf.getInt(
1328+
HConstants.MASTER_MERGE_DISPATCH_THREADS,
1329+
HConstants.MASTER_MERGE_DISPATCH_THREADS_DEFAULT));
13271330

13281331
// We depend on there being only one instance of this executor running
13291332
// at a time. To do concurrency, would need fencing of enable/disable of

hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -209,14 +209,21 @@ public List<NormalizationPlan> computePlansForTable(final TableName table) {
209209
ctx.getTableRegions().size());
210210

211211
final List<NormalizationPlan> plans = new ArrayList<>();
212+
int splitPlansCount = 0;
212213
if (proceedWithSplitPlanning) {
213-
plans.addAll(computeSplitNormalizationPlans(ctx));
214+
List<NormalizationPlan> splitPlans = computeSplitNormalizationPlans(ctx);
215+
splitPlansCount = splitPlans.size();
216+
plans.addAll(splitPlans);
214217
}
218+
int mergePlansCount = 0;
215219
if (proceedWithMergePlanning) {
216-
plans.addAll(computeMergeNormalizationPlans(ctx));
220+
List<NormalizationPlan> mergePlans = computeMergeNormalizationPlans(ctx);
221+
mergePlansCount = mergePlans.size();
222+
plans.addAll(mergePlans);
217223
}
218224

219-
LOG.debug("Computed {} normalization plans for table {}", plans.size(), table);
225+
LOG.debug("Computed normalization plans for table {}. Total plans: {}, split plans: {}, " +
226+
"merge plans: {}", table, plans.size(), splitPlansCount, mergePlansCount);
220227
return plans;
221228
}
222229

0 commit comments

Comments
 (0)