huaxiangsun commented on a change in pull request #1786:
URL: https://github.com/apache/hbase/pull/1786#discussion_r431383688



##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
##########
@@ -400,6 +401,8 @@ public void run() {
   private final LockManager lockManager = new LockManager(this);
 
   private RSGroupBasedLoadBalancer balancer;
+  // a lock to prevent concurrent normalization actions.
+  private final ReentrantLock normalizationInProgressLock = new 
ReentrantLock();

Review comment:
       It can also run from cli.

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
##########
@@ -1911,43 +1912,51 @@ public boolean normalizeRegions() throws IOException {
       return false;
     }
 
-    synchronized (this.normalizer) {
+    if (!normalizationInProgressLock.tryLock()) {
       // Don't run the normalizer concurrently
+      LOG.info("Normalization already in progress. Skipping request.");
+    } else {
+      try {
+        List<TableName> allEnabledTables = new ArrayList<>(
+          tableStateManager.getTablesInStates(TableState.State.ENABLED));
+        Collections.shuffle(allEnabledTables);
 
-      List<TableName> allEnabledTables = new ArrayList<>(
-        this.tableStateManager.getTablesInStates(TableState.State.ENABLED));
-
-      Collections.shuffle(allEnabledTables);
-
-      for (TableName table : allEnabledTables) {
-        TableDescriptor tblDesc = getTableDescriptors().get(table);
-        if (table.isSystemTable() || (tblDesc != null &&
-            !tblDesc.isNormalizationEnabled())) {
-          LOG.trace("Skipping normalization for {}, as it's either system"
-              + " table or doesn't have auto normalization turned on", table);
-          continue;
-        }
+        for (TableName table : allEnabledTables) {
+          if (table.isSystemTable()) {
+            continue;
+          }
+          final TableDescriptor tblDesc = getTableDescriptors().get(table);
+          if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+            LOG.debug(
+              "Skipping {} because normalization is disabled in its table 
properties.", table);
+            continue;
+          }
 
-        // make one last check that the cluster isn't shutting down before 
proceeding.
-        if (skipRegionManagementAction("region normalizer")) {
-          return false;
-        }
+          // make one last check that the cluster isn't shutting down before 
proceeding.
+          if (skipRegionManagementAction("region normalizer")) {
+            return false;
+          }
 
-        final List<NormalizationPlan> plans = 
this.normalizer.computePlanForTable(table);
-        if (CollectionUtils.isEmpty(plans)) {
-          return true;
-        }
+          final List<NormalizationPlan> plans = 
normalizer.computePlansForTable(table);
+          if (CollectionUtils.isEmpty(plans)) {
+            return true;
+          }
 
-        try (final Admin admin = 
asyncClusterConnection.toConnection().getAdmin()) {
-          for (NormalizationPlan plan : plans) {
-            plan.execute(admin);
-            if (plan.getType() == PlanType.SPLIT) {
-              splitPlanCount++;
-            } else if (plan.getType() == PlanType.MERGE) {
-              mergePlanCount++;
+          try (final Admin admin = 
asyncClusterConnection.toConnection().getAdmin()) {

Review comment:
       Sorry, this is an enhancement, can 
   final Admin admin = asyncClusterConnection.toConnection().getAdmin() be 
moved to before 
           for (TableName table : allEnabledTables) {
   so it wont create a admin instance for each table?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -18,126 +17,436 @@
  */
 package org.apache.hadoop.hbase.master.normalizer;
 
+import java.io.IOException;
+import java.time.Instant;
+import java.time.Period;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
-
-import org.apache.hadoop.hbase.HBaseIOException;
+import java.util.Objects;
+import java.util.function.BooleanSupplier;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Size;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import 
org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 /**
  * Simple implementation of region normalizer. Logic in use:
  * <ol>
- * <li>Get all regions of a given table
- * <li>Get avg size S of each region (by total size of store files reported in 
RegionMetrics)
- * <li>Seek every single region one by one. If a region R0 is bigger than S * 
2, it is kindly
- * requested to split. Thereon evaluate the next region R1
- * <li>Otherwise, if R0 + R1 is smaller than S, R0 and R1 are kindly requested 
to merge. Thereon
- * evaluate the next region R2
- * <li>Otherwise, R1 is evaluated
+ *   <li>Get all regions of a given table</li>
+ *   <li>Get avg size S of the regions in the table (by total size of store 
files reported in
+ *     RegionMetrics)</li>
+ *   <li>For each region R0, if R0 is bigger than S * 2, it is kindly 
requested to split.</li>
+ *   <li>Otherwise, for the next region in the chain R1, if R0 + R1 is smaller 
then S, R0 and R1
+ *     are kindly requested to merge.</li>
+ * </ol>
+ * <p>
+ * The following parameters are configurable:

Review comment:
       We should, :)

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
##########
@@ -1911,43 +1912,51 @@ public boolean normalizeRegions() throws IOException {
       return false;
     }
 
-    synchronized (this.normalizer) {
+    if (!normalizationInProgressLock.tryLock()) {
       // Don't run the normalizer concurrently
+      LOG.info("Normalization already in progress. Skipping request.");
+    } else {
+      try {
+        List<TableName> allEnabledTables = new ArrayList<>(
+          tableStateManager.getTablesInStates(TableState.State.ENABLED));
+        Collections.shuffle(allEnabledTables);
 
-      List<TableName> allEnabledTables = new ArrayList<>(
-        this.tableStateManager.getTablesInStates(TableState.State.ENABLED));
-
-      Collections.shuffle(allEnabledTables);
-
-      for (TableName table : allEnabledTables) {
-        TableDescriptor tblDesc = getTableDescriptors().get(table);
-        if (table.isSystemTable() || (tblDesc != null &&
-            !tblDesc.isNormalizationEnabled())) {
-          LOG.trace("Skipping normalization for {}, as it's either system"
-              + " table or doesn't have auto normalization turned on", table);
-          continue;
-        }
+        for (TableName table : allEnabledTables) {
+          if (table.isSystemTable()) {
+            continue;
+          }
+          final TableDescriptor tblDesc = getTableDescriptors().get(table);
+          if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+            LOG.debug(
+              "Skipping {} because normalization is disabled in its table 
properties.", table);
+            continue;
+          }
 
-        // make one last check that the cluster isn't shutting down before 
proceeding.
-        if (skipRegionManagementAction("region normalizer")) {
-          return false;
-        }
+          // make one last check that the cluster isn't shutting down before 
proceeding.
+          if (skipRegionManagementAction("region normalizer")) {
+            return false;
+          }
 
-        final List<NormalizationPlan> plans = 
this.normalizer.computePlanForTable(table);
-        if (CollectionUtils.isEmpty(plans)) {
-          return true;
-        }
+          final List<NormalizationPlan> plans = 
normalizer.computePlansForTable(table);
+          if (CollectionUtils.isEmpty(plans)) {
+            return true;

Review comment:
       +1, that is the original issue. It should be "continue" here.

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -18,126 +17,436 @@
  */
 package org.apache.hadoop.hbase.master.normalizer;
 
+import java.io.IOException;
+import java.time.Instant;
+import java.time.Period;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
-
-import org.apache.hadoop.hbase.HBaseIOException;
+import java.util.Objects;
+import java.util.function.BooleanSupplier;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Size;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import 
org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 /**
  * Simple implementation of region normalizer. Logic in use:
  * <ol>
- * <li>Get all regions of a given table
- * <li>Get avg size S of each region (by total size of store files reported in 
RegionMetrics)
- * <li>Seek every single region one by one. If a region R0 is bigger than S * 
2, it is kindly
- * requested to split. Thereon evaluate the next region R1
- * <li>Otherwise, if R0 + R1 is smaller than S, R0 and R1 are kindly requested 
to merge. Thereon
- * evaluate the next region R2
- * <li>Otherwise, R1 is evaluated
+ *   <li>Get all regions of a given table</li>
+ *   <li>Get avg size S of the regions in the table (by total size of store 
files reported in
+ *     RegionMetrics)</li>
+ *   <li>For each region R0, if R0 is bigger than S * 2, it is kindly 
requested to split.</li>
+ *   <li>Otherwise, for the next region in the chain R1, if R0 + R1 is smaller 
then S, R0 and R1
+ *     are kindly requested to merge.</li>
+ * </ol>
+ * <p>
+ * The following parameters are configurable:
+ * <ol>
+ *   <li>Whether to split a region as part of normalization. Configuration:
+ *     {@value SPLIT_ENABLED_KEY}, default: {@value 
DEFAULT_SPLIT_ENABLED}.</li>
+ *   <li>Whether to merge a region as part of normalization. Configuration:
+ *     {@value MERGE_ENABLED_KEY}, default: {@value 
DEFAULT_MERGE_ENABLED}.</li>
+ *   <li>The minimum number of regions in a table to consider it for 
normalization. Configuration:
+ *     {@value MIN_REGION_COUNT_KEY}, default: {@value 
DEFAULT_MIN_REGION_COUNT}.</li>
+ *   <li>The minimum age for a region to be considered for a merge, in days. 
Configuration:
+ *     {@value MERGE_MIN_REGION_AGE_DAYS_KEY}, default:
+ *     {@value DEFAULT_MERGE_MIN_REGION_AGE_DAYS}.</li>
+ *   <li>The minimum size for a region to be considered for a merge, in whole 
MBs. Configuration:
+ *     {@value MERGE_MIN_REGION_SIZE_MB_KEY}, default:
+ *     {@value DEFAULT_MERGE_MIN_REGION_SIZE_MB}.</li>
  * </ol>
  * <p>
- * Region sizes are coarse and approximate on the order of megabytes. 
Additionally, "empty" regions
- * (less than 1MB, with the previous note) are not merged away. This is by 
design to prevent
- * normalization from undoing the pre-splitting of a table.
+ * To see detailed logging of the application of these configuration values, 
set the log level for
+ * this class to `TRACE`.

Review comment:
       +1
   

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitPlanFirstComparator.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.hadoop.hbase.master.normalizer;
+
+import java.util.Comparator;
+
+/**
+ * Comparator class that gives higher priority to {@link 
SplitNormalizationPlan}.
+ */
+class SplitPlanFirstComparator implements Comparator<NormalizationPlan> {

Review comment:
       This is for sorting plans within one table, does not seem it is 
necessary though.

##########
File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
##########
@@ -69,517 +78,347 @@
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestSimpleRegionNormalizer.class);
 
-  private static final Logger LOG = 
LoggerFactory.getLogger(TestSimpleRegionNormalizer.class);
-
-  private RegionNormalizer normalizer;
+  private Configuration conf;
+  private SimpleRegionNormalizer normalizer;
   private MasterServices masterServices;
 
   @Rule
   public TestName name = new TestName();
 
-  @Test
-  public void testPlanComparator() {
-    Comparator<NormalizationPlan> comparator = new 
SimpleRegionNormalizer.PlanComparator();
-    NormalizationPlan splitPlan1 = new SplitNormalizationPlan(null, null);
-    NormalizationPlan splitPlan2 = new SplitNormalizationPlan(null, null);
-    NormalizationPlan mergePlan1 = new MergeNormalizationPlan(null, null);
-    NormalizationPlan mergePlan2 = new MergeNormalizationPlan(null, null);
-
-    assertEquals(0, comparator.compare(splitPlan1, splitPlan2));
-    assertEquals(0, comparator.compare(splitPlan2, splitPlan1));
-    assertEquals(0, comparator.compare(mergePlan1, mergePlan2));
-    assertEquals(0, comparator.compare(mergePlan2, mergePlan1));
-    assertTrue(comparator.compare(splitPlan1, mergePlan1) < 0);
-    assertTrue(comparator.compare(mergePlan1, splitPlan1) > 0);
+  @Before
+  public void before() {
+    conf = HBaseConfiguration.create();
   }
 
   @Test
-  public void testNoNormalizationForMetaTable() throws HBaseIOException {
+  public void testNoNormalizationForMetaTable() {
     TableName testTable = TableName.META_TABLE_NAME;
     List<RegionInfo> RegionInfo = new ArrayList<>();
     Map<byte[], Integer> regionSizes = new HashMap<>();
 
     setupMocksForNormalizer(regionSizes, RegionInfo);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable);
-    assertNull(plans);
+    List<NormalizationPlan> plans = normalizer.computePlansForTable(testTable);
+    assertThat(plans, empty());
   }
 
   @Test
-  public void testNoNormalizationIfTooFewRegions() throws HBaseIOException {
+  public void testNoNormalizationIfTooFewRegions() {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    List<RegionInfo> RegionInfo = new ArrayList<>();
-    Map<byte[], Integer> regionSizes = new HashMap<>();
-    RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("aaa"))
-        .setEndKey(Bytes.toBytes("bbb"))
-        .build();
-    RegionInfo.add(hri1);
-    regionSizes.put(hri1.getRegionName(), 10);
-
-    RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("bbb"))
-        .setEndKey(Bytes.toBytes("ccc"))
-        .build();
-    RegionInfo.add(hri2);
-    regionSizes.put(hri2.getRegionName(), 15);
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 2);
+    final Map<byte[], Integer> regionSizes = createRegionSizesMap(regionInfos, 
10, 15);
+    setupMocksForNormalizer(regionSizes, regionInfos);
 
-    setupMocksForNormalizer(regionSizes, RegionInfo);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
-    assertNull(plans);
+    List<NormalizationPlan> plans = normalizer.computePlansForTable(tableName);
+    assertThat(plans, empty());
   }
 
   @Test
-  public void testNoNormalizationOnNormalizedCluster() throws HBaseIOException 
{
+  public void testNoNormalizationOnNormalizedCluster() {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    List<RegionInfo> RegionInfo = new ArrayList<>();
-    Map<byte[], Integer> regionSizes = new HashMap<>();
-
-    RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("aaa"))
-        .setEndKey(Bytes.toBytes("bbb"))
-        .build();
-    RegionInfo.add(hri1);
-    regionSizes.put(hri1.getRegionName(), 10);
-
-    RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("bbb"))
-        .setEndKey(Bytes.toBytes("ccc"))
-        .build();
-    RegionInfo.add(hri2);
-    regionSizes.put(hri2.getRegionName(), 15);
-
-    RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("ccc"))
-        .setEndKey(Bytes.toBytes("ddd"))
-        .build();
-    RegionInfo.add(hri3);
-    regionSizes.put(hri3.getRegionName(), 8);
-
-    RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("ddd"))
-        .setEndKey(Bytes.toBytes("eee"))
-        .build();
-    regionSizes.put(hri4.getRegionName(), 10);
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 4);
+    final Map<byte[], Integer> regionSizes = createRegionSizesMap(regionInfos, 
10, 15, 8, 10);
+    setupMocksForNormalizer(regionSizes, regionInfos);
 
-    setupMocksForNormalizer(regionSizes, RegionInfo);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
-    assertNull(plans);
+    List<NormalizationPlan> plans = normalizer.computePlansForTable(tableName);
+    assertThat(plans, empty());
   }
 
-  private void noNormalizationOnTransitioningRegions(final RegionState.State 
state)
-    throws Exception {
+  private void noNormalizationOnTransitioningRegions(final RegionState.State 
state) {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    final List<RegionInfo> regionInfos = new LinkedList<>();
-    final Map<byte[], Integer> regionSizes = new HashMap<>();
-
-    final RegionInfo ri1 = RegionInfoBuilder.newBuilder(tableName)
-      .setStartKey(Bytes.toBytes("aaa"))
-      .setEndKey(Bytes.toBytes("bbb"))
-      .build();
-    regionInfos.add(ri1);
-    regionSizes.put(ri1.getRegionName(), 10);
-
-    final RegionInfo ri2 = RegionInfoBuilder.newBuilder(tableName)
-      .setStartKey(Bytes.toBytes("bbb"))
-      .setEndKey(Bytes.toBytes("ccc"))
-      .build();
-    regionInfos.add(ri2);
-    regionSizes.put(ri2.getRegionName(), 1);
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 3);
+    final Map<byte[], Integer> regionSizes = createRegionSizesMap(regionInfos, 
10, 1, 100);
 
     setupMocksForNormalizer(regionSizes, regionInfos);
     when(masterServices.getAssignmentManager().getRegionStates()
-      .getRegionState(any(RegionInfo.class))).thenReturn(
-      RegionState.createForTesting(null, state));
-    assertNull(
-      format("Unexpected plans for RegionState %s", state),
-      normalizer.computePlanForTable(tableName));
+      .getRegionState(any(RegionInfo.class)))
+      .thenReturn(RegionState.createForTesting(null, state));
+    assertThat(normalizer.getMinRegionCount(), 
greaterThanOrEqualTo(regionInfos.size()));
+
+    List<NormalizationPlan> plans = normalizer.computePlansForTable(tableName);
+    assertThat(format("Unexpected plans for RegionState %s", state), plans, 
empty());
   }
 
   @Test
-  public void testNoNormalizationOnMergingNewRegions() throws Exception {
+  public void testNoNormalizationOnMergingNewRegions() {
     noNormalizationOnTransitioningRegions(RegionState.State.MERGING_NEW);
   }
 
   @Test
-  public void testNoNormalizationOnMergingRegions() throws Exception {
+  public void testNoNormalizationOnMergingRegions() {
     noNormalizationOnTransitioningRegions(RegionState.State.MERGING);
   }
 
   @Test
-  public void testNoNormalizationOnMergedRegions() throws Exception {
+  public void testNoNormalizationOnMergedRegions() {
     noNormalizationOnTransitioningRegions(RegionState.State.MERGED);
   }
 
   @Test
-  public void testNoNormalizationOnSplittingNewRegions() throws Exception {
+  public void testNoNormalizationOnSplittingNewRegions() {
     noNormalizationOnTransitioningRegions(RegionState.State.SPLITTING_NEW);
   }
 
   @Test
-  public void testNoNormalizationOnSplittingRegions() throws Exception {
+  public void testNoNormalizationOnSplittingRegions() {
     noNormalizationOnTransitioningRegions(RegionState.State.SPLITTING);
   }
 
   @Test
-  public void testNoNormalizationOnSplitRegions() throws Exception {
+  public void testNoNormalizationOnSplitRegions() {
     noNormalizationOnTransitioningRegions(RegionState.State.SPLIT);
   }
 
   @Test
-  public void testMergeOfSmallRegions() throws HBaseIOException {
+  public void testMergeOfSmallRegions() {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    List<RegionInfo> RegionInfo = new ArrayList<>();
-    Map<byte[], Integer> regionSizes = new HashMap<>();
-
-    RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("aaa"))
-        .setEndKey(Bytes.toBytes("bbb"))
-        .build();
-    RegionInfo.add(hri1);
-    regionSizes.put(hri1.getRegionName(), 15);
-
-    RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("bbb"))
-        .setEndKey(Bytes.toBytes("ccc"))
-        .build();
-    RegionInfo.add(hri2);
-    regionSizes.put(hri2.getRegionName(), 5);
-
-    RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("ccc"))
-        .setEndKey(Bytes.toBytes("ddd"))
-        .build();
-    RegionInfo.add(hri3);
-    regionSizes.put(hri3.getRegionName(), 5);
-
-    RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("ddd"))
-        .setEndKey(Bytes.toBytes("eee"))
-        .build();
-    RegionInfo.add(hri4);
-    regionSizes.put(hri4.getRegionName(), 15);
-
-    RegionInfo hri5 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("eee"))
-        .setEndKey(Bytes.toBytes("fff"))
-        .build();
-    RegionInfo.add(hri5);
-    regionSizes.put(hri5.getRegionName(), 16);
-
-    setupMocksForNormalizer(regionSizes, RegionInfo);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 5);
+    final Map<byte[], Integer> regionSizes =
+      createRegionSizesMap(regionInfos, 15, 5, 5, 15, 16);
+    setupMocksForNormalizer(regionSizes, regionInfos);
 
-    NormalizationPlan plan = plans.get(0);
-    assertTrue(plan instanceof MergeNormalizationPlan);
-    assertEquals(hri2, ((MergeNormalizationPlan) plan).getFirstRegion());
-    assertEquals(hri3, ((MergeNormalizationPlan) plan).getSecondRegion());
+    List<NormalizationPlan> plans = normalizer.computePlansForTable(tableName);
+    assertThat(plans.get(0), instanceOf(MergeNormalizationPlan.class));
+    MergeNormalizationPlan plan = (MergeNormalizationPlan) plans.get(0);
+    assertEquals(regionInfos.get(1), plan.getFirstRegion());
+    assertEquals(regionInfos.get(2), plan.getSecondRegion());
   }
 
   // Test for situation illustrated in HBASE-14867
   @Test
-  public void testMergeOfSecondSmallestRegions() throws HBaseIOException {
+  public void testMergeOfSecondSmallestRegions() {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    List<RegionInfo> RegionInfo = new ArrayList<>();
-    Map<byte[], Integer> regionSizes = new HashMap<>();
-
-    RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("aaa"))
-        .setEndKey(Bytes.toBytes("bbb"))
-        .build();
-    RegionInfo.add(hri1);
-    regionSizes.put(hri1.getRegionName(), 1);
-
-    RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("bbb"))
-        .setEndKey(Bytes.toBytes("ccc"))
-        .build();
-    RegionInfo.add(hri2);
-    regionSizes.put(hri2.getRegionName(), 10000);
-
-    RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("ccc"))
-        .setEndKey(Bytes.toBytes("ddd"))
-        .build();
-    RegionInfo.add(hri3);
-    regionSizes.put(hri3.getRegionName(), 10000);
-
-    RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("ddd"))
-        .setEndKey(Bytes.toBytes("eee"))
-        .build();
-    RegionInfo.add(hri4);
-    regionSizes.put(hri4.getRegionName(), 10000);
-
-    RegionInfo hri5 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("eee"))
-        .setEndKey(Bytes.toBytes("fff"))
-        .build();
-    RegionInfo.add(hri5);
-    regionSizes.put(hri5.getRegionName(), 2700);
-
-    RegionInfo hri6 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("fff"))
-        .setEndKey(Bytes.toBytes("ggg"))
-        .build();
-    RegionInfo.add(hri6);
-    regionSizes.put(hri6.getRegionName(), 2700);
-
-    setupMocksForNormalizer(regionSizes, RegionInfo);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
-    NormalizationPlan plan = plans.get(0);
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 6);
+    final Map<byte[], Integer> regionSizes =
+      createRegionSizesMap(regionInfos, 1, 10000, 10000, 10000, 2700, 2700);
+    setupMocksForNormalizer(regionSizes, regionInfos);
 
-    assertTrue(plan instanceof MergeNormalizationPlan);
-    assertEquals(hri5, ((MergeNormalizationPlan) plan).getFirstRegion());
-    assertEquals(hri6, ((MergeNormalizationPlan) plan).getSecondRegion());
+    List<NormalizationPlan> plans = normalizer.computePlansForTable(tableName);
+    assertThat(plans.get(0), instanceOf(MergeNormalizationPlan.class));
+    MergeNormalizationPlan plan = (MergeNormalizationPlan) plans.get(0);
+    assertEquals(regionInfos.get(4), plan.getFirstRegion());
+    assertEquals(regionInfos.get(5), plan.getSecondRegion());
   }
 
   @Test
-  public void testMergeOfSmallNonAdjacentRegions() throws HBaseIOException {
+  public void testMergeOfSmallNonAdjacentRegions() {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    List<RegionInfo> RegionInfo = new ArrayList<>();
-    Map<byte[], Integer> regionSizes = new HashMap<>();
-
-    RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("aaa"))
-        .setEndKey(Bytes.toBytes("bbb"))
-        .build();
-    RegionInfo.add(hri1);
-    regionSizes.put(hri1.getRegionName(), 15);
-
-    RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("bbb"))
-        .setEndKey(Bytes.toBytes("ccc"))
-        .build();
-    RegionInfo.add(hri2);
-    regionSizes.put(hri2.getRegionName(), 5);
-
-    RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("ccc"))
-        .setEndKey(Bytes.toBytes("ddd"))
-        .build();
-    RegionInfo.add(hri3);
-    regionSizes.put(hri3.getRegionName(), 16);
-
-    RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("ddd"))
-        .setEndKey(Bytes.toBytes("eee"))
-        .build();
-    RegionInfo.add(hri4);
-    regionSizes.put(hri4.getRegionName(), 15);
-
-    RegionInfo hri5 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("ddd"))
-        .setEndKey(Bytes.toBytes("eee"))
-        .build();
-    RegionInfo.add(hri4);
-    regionSizes.put(hri5.getRegionName(), 5);
-
-    setupMocksForNormalizer(regionSizes, RegionInfo);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 5);
+    final Map<byte[], Integer> regionSizes =
+      createRegionSizesMap(regionInfos, 15, 5, 16, 15, 5);
+    setupMocksForNormalizer(regionSizes, regionInfos);
 
-    assertNull(plans);
+    List<NormalizationPlan> plans = normalizer.computePlansForTable(tableName);
+    assertThat(plans, empty());
   }
 
   @Test
-  public void testSplitOfLargeRegion() throws HBaseIOException {
+  public void testSplitOfLargeRegion() {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    List<RegionInfo> RegionInfo = new ArrayList<>();
-    Map<byte[], Integer> regionSizes = new HashMap<>();
-
-    RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("aaa"))
-        .setEndKey(Bytes.toBytes("bbb"))
-        .build();
-    RegionInfo.add(hri1);
-    regionSizes.put(hri1.getRegionName(), 8);
-
-    RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("bbb"))
-        .setEndKey(Bytes.toBytes("ccc"))
-        .build();
-    RegionInfo.add(hri2);
-    regionSizes.put(hri2.getRegionName(), 6);
-
-    RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("ccc"))
-        .setEndKey(Bytes.toBytes("ddd"))
-        .build();
-    RegionInfo.add(hri3);
-    regionSizes.put(hri3.getRegionName(), 10);
-
-    RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("ddd"))
-        .setEndKey(Bytes.toBytes("eee"))
-        .build();
-    RegionInfo.add(hri4);
-    regionSizes.put(hri4.getRegionName(), 30);
-
-    setupMocksForNormalizer(regionSizes, RegionInfo);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
-    NormalizationPlan plan = plans.get(0);
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 4);
+    final Map<byte[], Integer> regionSizes =
+      createRegionSizesMap(regionInfos, 8, 6, 10, 30);
+    setupMocksForNormalizer(regionSizes, regionInfos);
 
-    assertTrue(plan instanceof SplitNormalizationPlan);
-    assertEquals(hri4, ((SplitNormalizationPlan) plan).getRegionInfo());
+    List<NormalizationPlan> plans = normalizer.computePlansForTable(tableName);
+    assertThat(plans.get(0), instanceOf(SplitNormalizationPlan.class));
+    SplitNormalizationPlan plan = (SplitNormalizationPlan) plans.get(0);
+    assertEquals(regionInfos.get(3), plan.getRegionInfo());
   }
 
   @Test
   public void testSplitWithTargetRegionCount() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    List<RegionInfo> RegionInfo = new ArrayList<>();
-    Map<byte[], Integer> regionSizes = new HashMap<>();
-
-    RegionInfo hri1 = 
RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("aaa"))
-        .setEndKey(Bytes.toBytes("bbb")).build();
-    RegionInfo.add(hri1);
-    regionSizes.put(hri1.getRegionName(), 20);
-
-    RegionInfo hri2 = 
RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("bbb"))
-        .setEndKey(Bytes.toBytes("ccc")).build();
-    RegionInfo.add(hri2);
-    regionSizes.put(hri2.getRegionName(), 40);
-
-    RegionInfo hri3 = 
RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("ccc"))
-        .setEndKey(Bytes.toBytes("ddd")).build();
-    RegionInfo.add(hri3);
-    regionSizes.put(hri3.getRegionName(), 60);
-
-    RegionInfo hri4 = 
RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("ddd"))
-        .setEndKey(Bytes.toBytes("eee")).build();
-    RegionInfo.add(hri4);
-    regionSizes.put(hri4.getRegionName(), 80);
-
-    RegionInfo hri5 = 
RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("eee"))
-        .setEndKey(Bytes.toBytes("fff")).build();
-    RegionInfo.add(hri5);
-    regionSizes.put(hri5.getRegionName(), 100);
-
-    RegionInfo hri6 = 
RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("fff"))
-        .setEndKey(Bytes.toBytes("ggg")).build();
-    RegionInfo.add(hri6);
-    regionSizes.put(hri6.getRegionName(), 120);
-
-    setupMocksForNormalizer(regionSizes, RegionInfo);
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 6);
+    final Map<byte[], Integer> regionSizes =
+      createRegionSizesMap(regionInfos, 20, 40, 60, 80, 100, 120);
+    setupMocksForNormalizer(regionSizes, regionInfos);
 
     // test when target region size is 20
     
when(masterServices.getTableDescriptors().get(any()).getNormalizerTargetRegionSize())
         .thenReturn(20L);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
-    assertEquals(4, plans.size());
-
-    for (NormalizationPlan plan : plans) {
-      assertTrue(plan instanceof SplitNormalizationPlan);
-    }
+    List<NormalizationPlan> plans = normalizer.computePlansForTable(tableName);
+    assertThat(plans, iterableWithSize(4));
+    assertThat(plans, everyItem(instanceOf(SplitNormalizationPlan.class)));
 
     // test when target region size is 200
     
when(masterServices.getTableDescriptors().get(any()).getNormalizerTargetRegionSize())
         .thenReturn(200L);
-    plans = normalizer.computePlanForTable(tableName);
-    assertEquals(2, plans.size());
-    NormalizationPlan plan = plans.get(0);
-    assertTrue(plan instanceof MergeNormalizationPlan);
-    assertEquals(hri1, ((MergeNormalizationPlan) plan).getFirstRegion());
-    assertEquals(hri2, ((MergeNormalizationPlan) plan).getSecondRegion());
+    plans = normalizer.computePlansForTable(tableName);
+    assertThat(plans, iterableWithSize(2));
+    assertTrue(plans.get(0) instanceof MergeNormalizationPlan);
+    MergeNormalizationPlan plan = (MergeNormalizationPlan) plans.get(0);
+    assertEquals(regionInfos.get(0), plan.getFirstRegion());
+    assertEquals(regionInfos.get(1), plan.getSecondRegion());
   }
 
   @Test
   public void testSplitWithTargetRegionSize() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    List<RegionInfo> RegionInfo = new ArrayList<>();
-    Map<byte[], Integer> regionSizes = new HashMap<>();
-
-    RegionInfo hri1 = 
RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("aaa"))
-        .setEndKey(Bytes.toBytes("bbb")).build();
-    RegionInfo.add(hri1);
-    regionSizes.put(hri1.getRegionName(), 20);
-
-    RegionInfo hri2 = 
RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("bbb"))
-        .setEndKey(Bytes.toBytes("ccc")).build();
-    RegionInfo.add(hri2);
-    regionSizes.put(hri2.getRegionName(), 40);
-
-    RegionInfo hri3 = 
RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("ccc"))
-        .setEndKey(Bytes.toBytes("ddd")).build();
-    RegionInfo.add(hri3);
-    regionSizes.put(hri3.getRegionName(), 60);
-
-    RegionInfo hri4 = 
RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("ddd"))
-        .setEndKey(Bytes.toBytes("eee")).build();
-    RegionInfo.add(hri4);
-    regionSizes.put(hri4.getRegionName(), 80);
-
-    setupMocksForNormalizer(regionSizes, RegionInfo);
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 4);
+    final Map<byte[], Integer> regionSizes = createRegionSizesMap(regionInfos, 
20, 40, 60, 80);
+    setupMocksForNormalizer(regionSizes, regionInfos);
 
     // test when target region count is 8
     
when(masterServices.getTableDescriptors().get(any()).getNormalizerTargetRegionCount())
         .thenReturn(8);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
-    assertEquals(2, plans.size());
-
-    for (NormalizationPlan plan : plans) {
-      assertTrue(plan instanceof SplitNormalizationPlan);
-    }
+    List<NormalizationPlan> plans = normalizer.computePlansForTable(tableName);
+    assertThat(plans, iterableWithSize(2));
+    assertThat(plans, everyItem(instanceOf(SplitNormalizationPlan.class)));
 
     // test when target region count is 3
     
when(masterServices.getTableDescriptors().get(any()).getNormalizerTargetRegionCount())
         .thenReturn(3);
-    plans = normalizer.computePlanForTable(tableName);
-    assertEquals(1, plans.size());
-    NormalizationPlan plan = plans.get(0);
-    assertTrue(plan instanceof MergeNormalizationPlan);
-    assertEquals(hri1, ((MergeNormalizationPlan) plan).getFirstRegion());
-    assertEquals(hri2, ((MergeNormalizationPlan) plan).getSecondRegion());
+    plans = normalizer.computePlansForTable(tableName);
+    assertThat(plans, contains(instanceOf(MergeNormalizationPlan.class)));
+    MergeNormalizationPlan plan = (MergeNormalizationPlan) plans.get(0);
+    assertEquals(regionInfos.get(0), plan.getFirstRegion());
+    assertEquals(regionInfos.get(1), plan.getSecondRegion());
   }
 
   @Test
-  public void testSplitIfTooFewRegions() throws HBaseIOException {
+  public void testHonorsSplitEnabled() {
+    conf.setBoolean(SPLIT_ENABLED_KEY, true);
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    List<RegionInfo> RegionInfo = new ArrayList<>();
-    Map<byte[], Integer> regionSizes = new HashMap<>();
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 5);
+    final Map<byte[], Integer> regionSizes =
+      createRegionSizesMap(regionInfos, 5, 5, 20, 5, 5);
+    setupMocksForNormalizer(regionSizes, regionInfos);
+    assertThat(
+      normalizer.computePlansForTable(tableName),
+      contains(instanceOf(SplitNormalizationPlan.class)));
+
+    conf.setBoolean(SPLIT_ENABLED_KEY, false);
+    setupMocksForNormalizer(regionSizes, regionInfos);
+    assertThat(normalizer.computePlansForTable(tableName), empty());
+  }
+
+  @Test
+  public void testHonorsMergeEnabled() {
+    conf.setBoolean(MERGE_ENABLED_KEY, true);
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 5);
+    final Map<byte[], Integer> regionSizes =
+      createRegionSizesMap(regionInfos, 20, 5, 5, 20, 20);
+    setupMocksForNormalizer(regionSizes, regionInfos);
+    assertThat(
+      normalizer.computePlansForTable(tableName),
+      contains(instanceOf(MergeNormalizationPlan.class)));
 
-    RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("aaa"))
-        .setEndKey(Bytes.toBytes("bbb"))
-        .build();
-    RegionInfo.add(hri1);
-    regionSizes.put(hri1.getRegionName(), 1);
-
-    RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("bbb"))
-        .setEndKey(Bytes.toBytes("ccc"))
-        .build();
-    RegionInfo.add(hri2);
-    regionSizes.put(hri2.getRegionName(), 1);
-    // the third region is huge one
-    RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes("ccc"))
-        .setEndKey(Bytes.toBytes("ddd"))
-        .build();
-    RegionInfo.add(hri3);
-    regionSizes.put(hri3.getRegionName(), 10);
+    conf.setBoolean(MERGE_ENABLED_KEY, false);
+    setupMocksForNormalizer(regionSizes, regionInfos);
+    assertThat(normalizer.computePlansForTable(tableName), empty());
+  }
 
-    setupMocksForNormalizer(regionSizes, RegionInfo);
+  @Test
+  public void testHonorsMinimumRegionCount() {
+    conf.setInt(MIN_REGION_COUNT_KEY, 1);
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 3);
+    // create a table topology that results in both a merge plan and a split 
plan. Assert that the
+    // merge is only created when the when the number of table regions is 
above the region count
+    // threshold, and that the split plan is create in both cases.
+    final Map<byte[], Integer> regionSizes = createRegionSizesMap(regionInfos, 
1, 1, 10);
+    setupMocksForNormalizer(regionSizes, regionInfos);
+
+    List<NormalizationPlan> plans = normalizer.computePlansForTable(tableName);
+    assertThat(plans, contains(
+      instanceOf(SplitNormalizationPlan.class),
+      instanceOf(MergeNormalizationPlan.class)));
+    SplitNormalizationPlan splitPlan = (SplitNormalizationPlan) plans.get(0);
+    assertEquals(regionInfos.get(2), splitPlan.getRegionInfo());
+    MergeNormalizationPlan mergePlan = (MergeNormalizationPlan) plans.get(1);
+    assertEquals(regionInfos.get(0), mergePlan.getFirstRegion());
+    assertEquals(regionInfos.get(1), mergePlan.getSecondRegion());
+
+    // have to call setupMocks again because we don't have dynamic config 
update on normalizer.
+    conf.setInt(MIN_REGION_COUNT_KEY, 4);
+    setupMocksForNormalizer(regionSizes, regionInfos);
+    plans = normalizer.computePlansForTable(tableName);
+    assertThat(plans, contains(instanceOf(SplitNormalizationPlan.class)));
+    splitPlan = (SplitNormalizationPlan) plans.get(0);
+    assertEquals(regionInfos.get(2), splitPlan.getRegionInfo());
+  }
+
+  @Test
+  public void testHonorsMergeMinRegionAge() {
+    conf.setInt(MERGE_MIN_REGION_AGE_DAYS_KEY, 7);
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 4);
+    final Map<byte[], Integer> regionSizes =
+      createRegionSizesMap(regionInfos, 1, 1, 10, 10);
+    setupMocksForNormalizer(regionSizes, regionInfos);
+    assertEquals(Period.ofDays(7), normalizer.getMergeMinRegionAge());
+    assertThat(
+      normalizer.computePlansForTable(tableName),
+      everyItem(not(instanceOf(MergeNormalizationPlan.class))));
+
+    // have to call setupMocks again because we don't have dynamic config 
update on normalizer.
+    conf.unset(MERGE_MIN_REGION_AGE_DAYS_KEY);
+    setupMocksForNormalizer(regionSizes, regionInfos);
+    assertEquals(
+      Period.ofDays(DEFAULT_MERGE_MIN_REGION_AGE_DAYS), 
normalizer.getMergeMinRegionAge());
+    final List<NormalizationPlan> plans = 
normalizer.computePlansForTable(tableName);
+    assertThat(plans, not(empty()));
+    assertThat(plans, everyItem(instanceOf(MergeNormalizationPlan.class)));
+  }
+
+  @Test
+  public void testHonorsMergeMinRegionSize() {
+    conf.setBoolean(SPLIT_ENABLED_KEY, false);
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final List<RegionInfo> regionInfos = createRegionInfos(tableName, 5);
+    final Map<byte[], Integer> regionSizes = createRegionSizesMap(regionInfos, 
1, 2, 0, 10, 10);
+    setupMocksForNormalizer(regionSizes, regionInfos);
+
+    assertFalse(normalizer.isSplitEnabled());
+    assertEquals(1, normalizer.getMergeMinRegionSizeMb());
+    final List<NormalizationPlan> plans = 
normalizer.computePlansForTable(tableName);
+    assertThat(plans, everyItem(instanceOf(MergeNormalizationPlan.class)));
+    assertThat(plans, iterableWithSize(1));
+    final MergeNormalizationPlan plan = (MergeNormalizationPlan) plans.get(0);
+    assertEquals(regionInfos.get(0), plan.getFirstRegion());
+    assertEquals(regionInfos.get(1), plan.getSecondRegion());
+
+    conf.setInt(MERGE_MIN_REGION_SIZE_MB_KEY, 3);
+    setupMocksForNormalizer(regionSizes, regionInfos);
+    assertEquals(3, normalizer.getMergeMinRegionSizeMb());
+    assertThat(normalizer.computePlansForTable(tableName), empty());
+  }
 
-    Configuration configuration = HBaseConfiguration.create();
-    
configuration.setInt(AbstractRegionNormalizer.HBASE_REGION_NORMALIZER_MIN_REGION_COUNT_KEY,
 4);
-    when(masterServices.getConfiguration()).thenReturn(configuration);
+  // This test is to make sure that normalizer is only going to merge adjacent 
regions.
+  @Test
+  public void testNormalizerCannotMergeNonAdjacentRegions() {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    // create 5 regions with sizes to trigger merge of small regions. region 
ranges are:
+    // [, "aa"), ["aa", "aa1"), ["aa1", "aa1!"), ["aa1!", "aa2"), ["aa2", )
+    // Region ["aa", "aa1") and ["aa1!", "aa2") are not adjacent, they are not 
supposed to
+    // merged.
+    final byte[][] keys = {

Review comment:
       That is a nice move, thank you! I tested without the patch(sort based 
regioninfo in getMergePlan()), it also passed for me. Debugged a bit, 
regionInfos() here is already sorted based on STARTKEY. We need to simulate 
what is the order in master, so changing the keys to 
   `    final byte[][] keys = {
         null,
         Bytes.toBytes("aa"),
         Bytes.toBytes("aa1!"),
         Bytes.toBytes("aa1"),
         Bytes.toBytes("aa2"),
         null,
       };`
   will fix it. The regionInfo is already key based, the order does not matter.

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -18,126 +17,436 @@
  */
 package org.apache.hadoop.hbase.master.normalizer;
 
+import java.io.IOException;
+import java.time.Instant;
+import java.time.Period;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
-
-import org.apache.hadoop.hbase.HBaseIOException;
+import java.util.Objects;
+import java.util.function.BooleanSupplier;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Size;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import 
org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 /**
  * Simple implementation of region normalizer. Logic in use:
  * <ol>
- * <li>Get all regions of a given table
- * <li>Get avg size S of each region (by total size of store files reported in 
RegionMetrics)
- * <li>Seek every single region one by one. If a region R0 is bigger than S * 
2, it is kindly
- * requested to split. Thereon evaluate the next region R1
- * <li>Otherwise, if R0 + R1 is smaller than S, R0 and R1 are kindly requested 
to merge. Thereon
- * evaluate the next region R2
- * <li>Otherwise, R1 is evaluated
+ *   <li>Get all regions of a given table</li>
+ *   <li>Get avg size S of the regions in the table (by total size of store 
files reported in
+ *     RegionMetrics)</li>
+ *   <li>For each region R0, if R0 is bigger than S * 2, it is kindly 
requested to split.</li>
+ *   <li>Otherwise, for the next region in the chain R1, if R0 + R1 is smaller 
then S, R0 and R1
+ *     are kindly requested to merge.</li>
+ * </ol>
+ * <p>
+ * The following parameters are configurable:
+ * <ol>
+ *   <li>Whether to split a region as part of normalization. Configuration:
+ *     {@value SPLIT_ENABLED_KEY}, default: {@value 
DEFAULT_SPLIT_ENABLED}.</li>
+ *   <li>Whether to merge a region as part of normalization. Configuration:
+ *     {@value MERGE_ENABLED_KEY}, default: {@value 
DEFAULT_MERGE_ENABLED}.</li>
+ *   <li>The minimum number of regions in a table to consider it for 
normalization. Configuration:
+ *     {@value MIN_REGION_COUNT_KEY}, default: {@value 
DEFAULT_MIN_REGION_COUNT}.</li>
+ *   <li>The minimum age for a region to be considered for a merge, in days. 
Configuration:
+ *     {@value MERGE_MIN_REGION_AGE_DAYS_KEY}, default:
+ *     {@value DEFAULT_MERGE_MIN_REGION_AGE_DAYS}.</li>
+ *   <li>The minimum size for a region to be considered for a merge, in whole 
MBs. Configuration:
+ *     {@value MERGE_MIN_REGION_SIZE_MB_KEY}, default:
+ *     {@value DEFAULT_MERGE_MIN_REGION_SIZE_MB}.</li>
  * </ol>
  * <p>
- * Region sizes are coarse and approximate on the order of megabytes. 
Additionally, "empty" regions
- * (less than 1MB, with the previous note) are not merged away. This is by 
design to prevent
- * normalization from undoing the pre-splitting of a table.
+ * To see detailed logging of the application of these configuration values, 
set the log level for
+ * this class to `TRACE`.
  */
-@InterfaceAudience.Private
-public class SimpleRegionNormalizer extends AbstractRegionNormalizer {
-
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class SimpleRegionNormalizer implements RegionNormalizer {
   private static final Logger LOG = 
LoggerFactory.getLogger(SimpleRegionNormalizer.class);
-  private static long[] skippedCount = new 
long[NormalizationPlan.PlanType.values().length];
+
+  static final String SPLIT_ENABLED_KEY = "hbase.normalizer.split.enabled";
+  static final boolean DEFAULT_SPLIT_ENABLED = true;
+  static final String MERGE_ENABLED_KEY = "hbase.normalizer.merge.enabled";
+  static final boolean DEFAULT_MERGE_ENABLED = true;
+  // TODO: after HBASE-24416, `min.region.count` only applies to merge plans; 
should
+  //  deprecate/rename the configuration key.
+  static final String MIN_REGION_COUNT_KEY = 
"hbase.normalizer.min.region.count";
+  static final int DEFAULT_MIN_REGION_COUNT = 3;
+  static final String MERGE_MIN_REGION_AGE_DAYS_KEY = 
"hbase.normalizer.merge.min_region_age.days";
+  static final int DEFAULT_MERGE_MIN_REGION_AGE_DAYS = 3;
+  static final String MERGE_MIN_REGION_SIZE_MB_KEY = 
"hbase.normalizer.merge.min_region_size.mb";
+  static final int DEFAULT_MERGE_MIN_REGION_SIZE_MB = 1;
+
+  private final long[] skippedCount = new 
long[NormalizationPlan.PlanType.values().length];
+  private final Comparator<NormalizationPlan> planComparator = new 
SplitPlanFirstComparator();
+
+  private Configuration conf;
+  private MasterServices masterServices;
+  private boolean splitEnabled;
+  private boolean mergeEnabled;
+  private int minRegionCount;
+  private Period mergeMinRegionAge;
+  private int mergeMinRegionSizeMb;
+
+  public SimpleRegionNormalizer() {
+    splitEnabled = DEFAULT_SPLIT_ENABLED;
+    mergeEnabled = DEFAULT_MERGE_ENABLED;
+    minRegionCount = DEFAULT_MIN_REGION_COUNT;
+    mergeMinRegionAge = Period.ofDays(DEFAULT_MERGE_MIN_REGION_AGE_DAYS);
+    mergeMinRegionSizeMb = DEFAULT_MERGE_MIN_REGION_SIZE_MB;
+  }
 
   @Override
-  public void planSkipped(RegionInfo hri, PlanType type) {
-    skippedCount[type.ordinal()]++;
+  public Configuration getConf() {
+    return conf;
   }
 
   @Override
-  public long getSkippedCount(NormalizationPlan.PlanType type) {
-    return skippedCount[type.ordinal()];
+  public void setConf(final Configuration conf) {
+    if (conf == null) {
+      return;
+    }
+    this.conf = conf;
+    splitEnabled = conf.getBoolean(SPLIT_ENABLED_KEY, DEFAULT_SPLIT_ENABLED);
+    mergeEnabled = conf.getBoolean(MERGE_ENABLED_KEY, DEFAULT_MERGE_ENABLED);
+    minRegionCount = parseMinRegionCount(conf);
+    mergeMinRegionAge = parseMergeMinRegionAge(conf);
+    mergeMinRegionSizeMb = parseMergeMinRegionSizeMb(conf);
+  }
+
+  private static int parseMinRegionCount(final Configuration conf) {
+    final int parsedValue = conf.getInt(MIN_REGION_COUNT_KEY, 
DEFAULT_MIN_REGION_COUNT);
+    final int settledValue = Math.max(1, parsedValue);
+    if (parsedValue != settledValue) {
+      warnInvalidValue(MIN_REGION_COUNT_KEY, parsedValue, settledValue);
+    }
+    return settledValue;
+  }
+
+  private static Period parseMergeMinRegionAge(final Configuration conf) {
+    final int parsedValue =
+      conf.getInt(MERGE_MIN_REGION_AGE_DAYS_KEY, 
DEFAULT_MERGE_MIN_REGION_AGE_DAYS);
+    final int settledValue = Math.max(0, parsedValue);
+    if (parsedValue != settledValue) {
+      warnInvalidValue(MERGE_MIN_REGION_AGE_DAYS_KEY, parsedValue, 
settledValue);
+    }
+    return Period.ofDays(settledValue);
+  }
+
+  private static int parseMergeMinRegionSizeMb(final Configuration conf) {
+    final int parsedValue =
+      conf.getInt(MERGE_MIN_REGION_SIZE_MB_KEY, 
DEFAULT_MERGE_MIN_REGION_SIZE_MB);
+    final int settledValue = Math.max(0, parsedValue);
+    if (parsedValue != settledValue) {
+      warnInvalidValue(MERGE_MIN_REGION_SIZE_MB_KEY, parsedValue, 
settledValue);
+    }
+    return settledValue;
+  }
+
+  private static <T> void warnInvalidValue(final String key, final T 
parsedValue,
+    final T settledValue) {
+    LOG.warn("Configured value {}={} is invalid. Setting value to {}.",
+      key, parsedValue, settledValue);
   }
 
   /**
-   * Comparator class that gives higher priority to region Split plan.
+   * Return this instance's configured value for {@value SPLIT_ENABLED_KEY}.
    */
-  static class PlanComparator implements Comparator<NormalizationPlan> {
-    @Override
-    public int compare(NormalizationPlan plan1, NormalizationPlan plan2) {
-      boolean plan1IsSplit = plan1 instanceof SplitNormalizationPlan;
-      boolean plan2IsSplit = plan2 instanceof SplitNormalizationPlan;
-      if (plan1IsSplit && plan2IsSplit) {
-        return 0;
-      } else if (plan1IsSplit) {
-        return -1;
-      } else if (plan2IsSplit) {
-        return 1;
-      } else {
-        return 0;
-      }
-    }
+  public boolean isSplitEnabled() {
+    return splitEnabled;
   }
 
-  private Comparator<NormalizationPlan> planComparator = new PlanComparator();
+  /**
+   * Return this instance's configured value for {@value MERGE_ENABLED_KEY}.
+   */
+  public boolean isMergeEnabled() {
+    return mergeEnabled;
+  }
 
   /**
-   * Computes next most "urgent" normalization action on the table. Action may 
be either a split, or
-   * a merge, or no action.
-   * @param table table to normalize
-   * @return normalization plan to execute
+   * Return this instance's configured value for {@value MIN_REGION_COUNT_KEY}.
+   */
+  public int getMinRegionCount() {
+    return minRegionCount;
+  }
+
+  /**
+   * Return this instance's configured value for {@value 
MERGE_MIN_REGION_AGE_DAYS_KEY}.
    */
+  public Period getMergeMinRegionAge() {
+    return mergeMinRegionAge;
+  }
+
+  /**
+   * Return this instance's configured value for {@value 
MERGE_MIN_REGION_SIZE_MB_KEY}.
+   */
+  public int getMergeMinRegionSizeMb() {
+    return mergeMinRegionSizeMb;
+  }
+
   @Override
-  public List<NormalizationPlan> computePlanForTable(TableName table) throws 
HBaseIOException {
+  public void setMasterServices(final MasterServices masterServices) {
+    this.masterServices = masterServices;
+  }
+
+  @Override
+  public void planSkipped(final RegionInfo hri, final PlanType type) {
+    skippedCount[type.ordinal()]++;
+  }
+
+  @Override
+  public long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  @Override
+  public List<NormalizationPlan> computePlansForTable(TableName table) {
     if (table == null || table.isSystemTable()) {
       LOG.debug("Normalization of system table {} isn't allowed", table);
-      return null;
+      return Collections.emptyList();
     }
-    boolean splitEnabled = isSplitEnabled();
-    boolean mergeEnabled = isMergeEnabled();
+    boolean splitEnabled = isSplitEnabled() && 
isMasterSwitchEnabled(MasterSwitchType.SPLIT);
+    boolean mergeEnabled = isMergeEnabled() && 
isMasterSwitchEnabled(MasterSwitchType.MERGE);
     if (!mergeEnabled && !splitEnabled) {
-      LOG.debug("Both split and merge are disabled for table: {}", table);
-      return null;
+      LOG.debug("Both split and merge are disabled. Skipping normalization of 
table: {}", table);
+      return Collections.emptyList();
     }
+
     List<NormalizationPlan> plans = new ArrayList<>();
-    List<RegionInfo> tableRegions =
-        
masterServices.getAssignmentManager().getRegionStates().getRegionsOfTable(table);
+    List<RegionInfo> tableRegions = masterServices.getAssignmentManager()
+      .getRegionStates()
+      .getRegionsOfTable(table);
 
-    if (tableRegions == null) {
-      return null;
+    if (CollectionUtils.isEmpty(tableRegions)) {
+      return Collections.emptyList();
     }
 
     LOG.debug("Computing normalization plan for table:  {}, number of regions: 
{}", table,
       tableRegions.size());
 
     if (splitEnabled) {
-      List<NormalizationPlan> splitPlans = getSplitNormalizationPlan(table);
-      if (splitPlans != null) {
-        plans.addAll(splitPlans);
+      plans.addAll(computeSplitNormalizationPlans(table));
+    }
+    if (mergeEnabled) {
+      plans.addAll(computeMergeNormalizationPlans(table));
+    }
+
+    plans.sort(planComparator);

Review comment:
       Seems this sort is unnecessary? All split plans are added to the head of 
the list, followed by merge plans. Sorry, this is the original code.

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -18,126 +17,436 @@
  */
 package org.apache.hadoop.hbase.master.normalizer;
 
+import java.io.IOException;
+import java.time.Instant;
+import java.time.Period;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
-
-import org.apache.hadoop.hbase.HBaseIOException;
+import java.util.Objects;
+import java.util.function.BooleanSupplier;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Size;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import 
org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 /**
  * Simple implementation of region normalizer. Logic in use:
  * <ol>
- * <li>Get all regions of a given table
- * <li>Get avg size S of each region (by total size of store files reported in 
RegionMetrics)
- * <li>Seek every single region one by one. If a region R0 is bigger than S * 
2, it is kindly
- * requested to split. Thereon evaluate the next region R1
- * <li>Otherwise, if R0 + R1 is smaller than S, R0 and R1 are kindly requested 
to merge. Thereon
- * evaluate the next region R2
- * <li>Otherwise, R1 is evaluated
+ *   <li>Get all regions of a given table</li>
+ *   <li>Get avg size S of the regions in the table (by total size of store 
files reported in
+ *     RegionMetrics)</li>
+ *   <li>For each region R0, if R0 is bigger than S * 2, it is kindly 
requested to split.</li>
+ *   <li>Otherwise, for the next region in the chain R1, if R0 + R1 is smaller 
then S, R0 and R1
+ *     are kindly requested to merge.</li>
+ * </ol>
+ * <p>
+ * The following parameters are configurable:
+ * <ol>
+ *   <li>Whether to split a region as part of normalization. Configuration:
+ *     {@value SPLIT_ENABLED_KEY}, default: {@value 
DEFAULT_SPLIT_ENABLED}.</li>
+ *   <li>Whether to merge a region as part of normalization. Configuration:
+ *     {@value MERGE_ENABLED_KEY}, default: {@value 
DEFAULT_MERGE_ENABLED}.</li>
+ *   <li>The minimum number of regions in a table to consider it for 
normalization. Configuration:
+ *     {@value MIN_REGION_COUNT_KEY}, default: {@value 
DEFAULT_MIN_REGION_COUNT}.</li>
+ *   <li>The minimum age for a region to be considered for a merge, in days. 
Configuration:
+ *     {@value MERGE_MIN_REGION_AGE_DAYS_KEY}, default:
+ *     {@value DEFAULT_MERGE_MIN_REGION_AGE_DAYS}.</li>
+ *   <li>The minimum size for a region to be considered for a merge, in whole 
MBs. Configuration:
+ *     {@value MERGE_MIN_REGION_SIZE_MB_KEY}, default:
+ *     {@value DEFAULT_MERGE_MIN_REGION_SIZE_MB}.</li>
  * </ol>
  * <p>
- * Region sizes are coarse and approximate on the order of megabytes. 
Additionally, "empty" regions
- * (less than 1MB, with the previous note) are not merged away. This is by 
design to prevent
- * normalization from undoing the pre-splitting of a table.
+ * To see detailed logging of the application of these configuration values, 
set the log level for
+ * this class to `TRACE`.
  */
-@InterfaceAudience.Private
-public class SimpleRegionNormalizer extends AbstractRegionNormalizer {
-
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class SimpleRegionNormalizer implements RegionNormalizer {
   private static final Logger LOG = 
LoggerFactory.getLogger(SimpleRegionNormalizer.class);
-  private static long[] skippedCount = new 
long[NormalizationPlan.PlanType.values().length];
+
+  static final String SPLIT_ENABLED_KEY = "hbase.normalizer.split.enabled";
+  static final boolean DEFAULT_SPLIT_ENABLED = true;
+  static final String MERGE_ENABLED_KEY = "hbase.normalizer.merge.enabled";
+  static final boolean DEFAULT_MERGE_ENABLED = true;
+  // TODO: after HBASE-24416, `min.region.count` only applies to merge plans; 
should
+  //  deprecate/rename the configuration key.
+  static final String MIN_REGION_COUNT_KEY = 
"hbase.normalizer.min.region.count";
+  static final int DEFAULT_MIN_REGION_COUNT = 3;
+  static final String MERGE_MIN_REGION_AGE_DAYS_KEY = 
"hbase.normalizer.merge.min_region_age.days";
+  static final int DEFAULT_MERGE_MIN_REGION_AGE_DAYS = 3;
+  static final String MERGE_MIN_REGION_SIZE_MB_KEY = 
"hbase.normalizer.merge.min_region_size.mb";
+  static final int DEFAULT_MERGE_MIN_REGION_SIZE_MB = 1;
+
+  private final long[] skippedCount = new 
long[NormalizationPlan.PlanType.values().length];
+  private final Comparator<NormalizationPlan> planComparator = new 
SplitPlanFirstComparator();
+
+  private Configuration conf;
+  private MasterServices masterServices;
+  private boolean splitEnabled;
+  private boolean mergeEnabled;
+  private int minRegionCount;
+  private Period mergeMinRegionAge;
+  private int mergeMinRegionSizeMb;
+
+  public SimpleRegionNormalizer() {
+    splitEnabled = DEFAULT_SPLIT_ENABLED;
+    mergeEnabled = DEFAULT_MERGE_ENABLED;
+    minRegionCount = DEFAULT_MIN_REGION_COUNT;
+    mergeMinRegionAge = Period.ofDays(DEFAULT_MERGE_MIN_REGION_AGE_DAYS);
+    mergeMinRegionSizeMb = DEFAULT_MERGE_MIN_REGION_SIZE_MB;
+  }
 
   @Override
-  public void planSkipped(RegionInfo hri, PlanType type) {
-    skippedCount[type.ordinal()]++;
+  public Configuration getConf() {
+    return conf;
   }
 
   @Override
-  public long getSkippedCount(NormalizationPlan.PlanType type) {
-    return skippedCount[type.ordinal()];
+  public void setConf(final Configuration conf) {
+    if (conf == null) {
+      return;
+    }
+    this.conf = conf;
+    splitEnabled = conf.getBoolean(SPLIT_ENABLED_KEY, DEFAULT_SPLIT_ENABLED);
+    mergeEnabled = conf.getBoolean(MERGE_ENABLED_KEY, DEFAULT_MERGE_ENABLED);
+    minRegionCount = parseMinRegionCount(conf);
+    mergeMinRegionAge = parseMergeMinRegionAge(conf);
+    mergeMinRegionSizeMb = parseMergeMinRegionSizeMb(conf);
+  }
+
+  private static int parseMinRegionCount(final Configuration conf) {
+    final int parsedValue = conf.getInt(MIN_REGION_COUNT_KEY, 
DEFAULT_MIN_REGION_COUNT);
+    final int settledValue = Math.max(1, parsedValue);
+    if (parsedValue != settledValue) {
+      warnInvalidValue(MIN_REGION_COUNT_KEY, parsedValue, settledValue);
+    }
+    return settledValue;
+  }
+
+  private static Period parseMergeMinRegionAge(final Configuration conf) {
+    final int parsedValue =
+      conf.getInt(MERGE_MIN_REGION_AGE_DAYS_KEY, 
DEFAULT_MERGE_MIN_REGION_AGE_DAYS);
+    final int settledValue = Math.max(0, parsedValue);
+    if (parsedValue != settledValue) {
+      warnInvalidValue(MERGE_MIN_REGION_AGE_DAYS_KEY, parsedValue, 
settledValue);
+    }
+    return Period.ofDays(settledValue);
+  }
+
+  private static int parseMergeMinRegionSizeMb(final Configuration conf) {
+    final int parsedValue =
+      conf.getInt(MERGE_MIN_REGION_SIZE_MB_KEY, 
DEFAULT_MERGE_MIN_REGION_SIZE_MB);
+    final int settledValue = Math.max(0, parsedValue);
+    if (parsedValue != settledValue) {
+      warnInvalidValue(MERGE_MIN_REGION_SIZE_MB_KEY, parsedValue, 
settledValue);
+    }
+    return settledValue;
+  }
+
+  private static <T> void warnInvalidValue(final String key, final T 
parsedValue,
+    final T settledValue) {
+    LOG.warn("Configured value {}={} is invalid. Setting value to {}.",
+      key, parsedValue, settledValue);
   }
 
   /**
-   * Comparator class that gives higher priority to region Split plan.
+   * Return this instance's configured value for {@value SPLIT_ENABLED_KEY}.
    */
-  static class PlanComparator implements Comparator<NormalizationPlan> {
-    @Override
-    public int compare(NormalizationPlan plan1, NormalizationPlan plan2) {
-      boolean plan1IsSplit = plan1 instanceof SplitNormalizationPlan;
-      boolean plan2IsSplit = plan2 instanceof SplitNormalizationPlan;
-      if (plan1IsSplit && plan2IsSplit) {
-        return 0;
-      } else if (plan1IsSplit) {
-        return -1;
-      } else if (plan2IsSplit) {
-        return 1;
-      } else {
-        return 0;
-      }
-    }
+  public boolean isSplitEnabled() {
+    return splitEnabled;
   }
 
-  private Comparator<NormalizationPlan> planComparator = new PlanComparator();
+  /**
+   * Return this instance's configured value for {@value MERGE_ENABLED_KEY}.
+   */
+  public boolean isMergeEnabled() {
+    return mergeEnabled;
+  }
 
   /**
-   * Computes next most "urgent" normalization action on the table. Action may 
be either a split, or
-   * a merge, or no action.
-   * @param table table to normalize
-   * @return normalization plan to execute
+   * Return this instance's configured value for {@value MIN_REGION_COUNT_KEY}.
+   */
+  public int getMinRegionCount() {
+    return minRegionCount;
+  }
+
+  /**
+   * Return this instance's configured value for {@value 
MERGE_MIN_REGION_AGE_DAYS_KEY}.
    */
+  public Period getMergeMinRegionAge() {
+    return mergeMinRegionAge;
+  }
+
+  /**
+   * Return this instance's configured value for {@value 
MERGE_MIN_REGION_SIZE_MB_KEY}.
+   */
+  public int getMergeMinRegionSizeMb() {
+    return mergeMinRegionSizeMb;
+  }
+
   @Override
-  public List<NormalizationPlan> computePlanForTable(TableName table) throws 
HBaseIOException {
+  public void setMasterServices(final MasterServices masterServices) {
+    this.masterServices = masterServices;
+  }
+
+  @Override
+  public void planSkipped(final RegionInfo hri, final PlanType type) {
+    skippedCount[type.ordinal()]++;
+  }
+
+  @Override
+  public long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  @Override
+  public List<NormalizationPlan> computePlansForTable(TableName table) {
     if (table == null || table.isSystemTable()) {
       LOG.debug("Normalization of system table {} isn't allowed", table);
-      return null;
+      return Collections.emptyList();
     }
-    boolean splitEnabled = isSplitEnabled();
-    boolean mergeEnabled = isMergeEnabled();
+    boolean splitEnabled = isSplitEnabled() && 
isMasterSwitchEnabled(MasterSwitchType.SPLIT);
+    boolean mergeEnabled = isMergeEnabled() && 
isMasterSwitchEnabled(MasterSwitchType.MERGE);
     if (!mergeEnabled && !splitEnabled) {
-      LOG.debug("Both split and merge are disabled for table: {}", table);
-      return null;
+      LOG.debug("Both split and merge are disabled. Skipping normalization of 
table: {}", table);
+      return Collections.emptyList();
     }
+
     List<NormalizationPlan> plans = new ArrayList<>();
-    List<RegionInfo> tableRegions =
-        
masterServices.getAssignmentManager().getRegionStates().getRegionsOfTable(table);
+    List<RegionInfo> tableRegions = masterServices.getAssignmentManager()
+      .getRegionStates()
+      .getRegionsOfTable(table);
 
-    if (tableRegions == null) {
-      return null;
+    if (CollectionUtils.isEmpty(tableRegions)) {
+      return Collections.emptyList();
     }
 
     LOG.debug("Computing normalization plan for table:  {}, number of regions: 
{}", table,
       tableRegions.size());
 
     if (splitEnabled) {
-      List<NormalizationPlan> splitPlans = getSplitNormalizationPlan(table);
-      if (splitPlans != null) {
-        plans.addAll(splitPlans);
+      plans.addAll(computeSplitNormalizationPlans(table));
+    }
+    if (mergeEnabled) {
+      plans.addAll(computeMergeNormalizationPlans(table));
+    }
+
+    plans.sort(planComparator);

Review comment:
       It can be addressed in a new jira, not necessary in this jira.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to