You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2016/01/19 21:36:01 UTC

[3/5] hbase git commit: HBASE-15073 Revert due to different opinion on usefulness

HBASE-15073 Revert due to different opinion on usefulness


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/eb17f74b
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/eb17f74b
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/eb17f74b

Branch: refs/heads/hbase-12439
Commit: eb17f74b9e45146a21584f44e7a811fedeee138f
Parents: 9bf26f4
Author: tedyu <yu...@gmail.com>
Authored: Mon Jan 18 17:51:21 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Mon Jan 18 17:51:21 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   | 60 +++++++-------------
 .../hbase/normalizer/NormalizationPlan.java     | 45 ---------------
 .../hadoop/hbase/master/AssignmentManager.java  |  2 +-
 .../org/apache/hadoop/hbase/master/HMaster.java | 23 +++-----
 .../normalizer/EmptyNormalizationPlan.java      |  2 +-
 .../normalizer/MergeNormalizationPlan.java      |  2 +-
 .../master/normalizer/NormalizationPlan.java    | 45 +++++++++++++++
 .../master/normalizer/RegionNormalizer.java     |  6 +-
 .../normalizer/SimpleRegionNormalizer.java      | 13 ++---
 .../normalizer/SplitNormalizationPlan.java      |  2 +-
 .../normalizer/TestSimpleRegionNormalizer.java  | 48 ++++------------
 .../TestSimpleRegionNormalizerOnCluster.java    |  6 +-
 hbase-shell/src/main/ruby/hbase/admin.rb        |  5 +-
 .../src/main/ruby/shell/commands/normalize.rb   |  2 +-
 .../ruby/shell/commands/normalizer_switch.rb    |  3 +-
 15 files changed, 103 insertions(+), 161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index 0fb0455..1bd4e07 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -40,8 +40,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema;
@@ -187,14 +185,13 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
 
   /**
    * <em>INTERNAL</em> Used by shell/rest interface to access this metadata
-   * attribute which denotes the allowed types of action (split/merge) when the table is treated
-   * by region normalizer.
+   * attribute which denotes if the table should be treated by region normalizer.
    *
-   * @see #getDesiredNormalizationTypes()
+   * @see #isNormalizationEnabled()
    */
-  public static final String NORMALIZATION_MODE = "NORMALIZATION_MODE";
-  private static final Bytes NORMALIZATION_MODE_KEY =
-    new Bytes(Bytes.toBytes(NORMALIZATION_MODE));
+  public static final String NORMALIZATION_ENABLED = "NORMALIZATION_ENABLED";
+  private static final Bytes NORMALIZATION_ENABLED_KEY =
+    new Bytes(Bytes.toBytes(NORMALIZATION_ENABLED));
 
   /** Default durability for HTD is USE_DEFAULT, which defaults to HBase-global default value */
   private static final Durability DEFAULT_DURABLITY = Durability.USE_DEFAULT;
@@ -223,6 +220,11 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
   public static final boolean DEFAULT_COMPACTION_ENABLED = true;
 
   /**
+   * Constant that denotes whether the table is normalized by default.
+   */
+  public static final boolean DEFAULT_NORMALIZATION_ENABLED = false;
+
+  /**
    * Constant that denotes the maximum default size of the memstore after which
    * the contents are flushed to the store files
    */
@@ -247,7 +249,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
         String.valueOf(DEFAULT_DEFERRED_LOG_FLUSH));
     DEFAULT_VALUES.put(DURABILITY, DEFAULT_DURABLITY.name()); //use the enum name
     DEFAULT_VALUES.put(REGION_REPLICATION, String.valueOf(DEFAULT_REGION_REPLICATION));
-    DEFAULT_VALUES.put(NORMALIZATION_MODE, "");
+    DEFAULT_VALUES.put(NORMALIZATION_ENABLED, String.valueOf(DEFAULT_NORMALIZATION_ENABLED));
     for (String s : DEFAULT_VALUES.keySet()) {
       RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(s)));
     }
@@ -638,42 +640,22 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
   }
 
   /**
-   * Check if normalization flag of the table. If flag is
-   * empty then region normalizer won't attempt to normalize this table.
+   * Check if normalization enable flag of the table is true. If flag is
+   * false then no region normalizer won't attempt to normalize this table.
    *
-   * @return List of PlanType if region normalization is enabled for this table
-   *         null means region normalization is disabled
+   * @return true if region normalization is enabled for this table
    */
-  public List<PlanType> getDesiredNormalizationTypes() {
-    byte [] value = getValue(NORMALIZATION_MODE_KEY);
-    if (value == null) {
-      return null;
-    }
-    String strValue = Bytes.toString(value);
-    if (strValue.isEmpty()) {
-      return null;
-    }
-    List<NormalizationPlan.PlanType> types = new ArrayList<>();
-    if (strValue.toUpperCase().contains("M")) {
-      types.add(PlanType.MERGE);
-    }
-    if (strValue.toUpperCase().contains("S")) {
-      types.add(PlanType.SPLIT);
-    }
-    return types;
+  public boolean isNormalizationEnabled() {
+    return isSomething(NORMALIZATION_ENABLED_KEY, DEFAULT_NORMALIZATION_ENABLED);
   }
 
   /**
-   * Setting the types of action for table normalization mode flag.
+   * Setting the table normalization enable flag.
    *
-   * @param types String containing desired types of action:
-   *        "M" for region merge
-   *        "S" for region split
-   *        "MS" for region merge / split
-   */
-  public HTableDescriptor setNormalizationMode(final String types) {
-    setValue(NORMALIZATION_MODE_KEY, types == null || types.isEmpty() ? null :
-      new Bytes(Bytes.toBytes(types.toUpperCase())));
+   * @param isEnable True if enable normalization.
+   */
+  public HTableDescriptor setNormalizationEnabled(final boolean isEnable) {
+    setValue(NORMALIZATION_ENABLED_KEY, isEnable ? TRUE : FALSE);
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-client/src/main/java/org/apache/hadoop/hbase/normalizer/NormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/normalizer/NormalizationPlan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/normalizer/NormalizationPlan.java
deleted file mode 100644
index 66481e6..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/normalizer/NormalizationPlan.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- *
- * 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.normalizer;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Admin;
-
-/**
- * Interface for normalization plan.
- */
-@InterfaceAudience.Private
-public interface NormalizationPlan {
-  enum PlanType {
-    SPLIT,
-    MERGE,
-    NONE
-  }
-
-  /**
-   * Executes normalization plan on cluster (does actual splitting/merging work).
-   * @param admin instance of Admin
-   */
-  void execute(Admin admin);
-
-  /**
-   * @return the type of this plan
-   */
-  PlanType getType();
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index 405ec65..b455828 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -73,7 +73,7 @@ import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
 import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer;
 import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
 import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.quotas.QuotaExceededException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 9f5e7e3..43f8efa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -93,6 +93,8 @@ import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
 import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
@@ -115,8 +117,6 @@ import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
@@ -1331,21 +1331,14 @@ public class HMaster extends HRegionServer implements MasterServices {
 
       for (TableName table : allEnabledTables) {
         TableDescriptor tblDesc = getTableDescriptors().getDescriptor(table);
-        if (table.isSystemTable()) {
-          LOG.debug("Skipping normalization for table: " + table + ", as it's system table");
+        if (table.isSystemTable() || (tblDesc != null &&
+            tblDesc.getHTableDescriptor() != null &&
+            !tblDesc.getHTableDescriptor().isNormalizationEnabled())) {
+          LOG.debug("Skipping normalization for table: " + table + ", as it's either system"
+              + " table or doesn't have auto normalization turned on");
           continue;
         }
-        List<PlanType> types = null;
-        if (tblDesc != null &&
-            tblDesc.getHTableDescriptor() != null) {
-          types = tblDesc.getHTableDescriptor().getDesiredNormalizationTypes();
-          if (types == null) {
-            LOG.debug("Skipping normalization for table: " + table + ", as it"
-                + " doesn't have auto normalization turned on");
-            continue;
-          }
-        }
-        List<NormalizationPlan> plans = this.normalizer.computePlanForTable(table, types);
+        List<NormalizationPlan> plans = this.normalizer.computePlanForTable(table);
         if (plans != null) {
           for (NormalizationPlan plan : plans) {
             plan.execute(clusterConnection.getAdmin());

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java
index 29cc0c3..5aecc48 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.master.normalizer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 
 /**
  * Plan which signifies that no normalization is required,

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
index f3ce1d5..e2035bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
@@ -23,7 +23,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java
new file mode 100644
index 0000000..9f866d3
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java
@@ -0,0 +1,45 @@
+/**
+ *
+ * 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+
+/**
+ * Interface for normalization plan.
+ */
+@InterfaceAudience.Private
+public interface NormalizationPlan {
+  enum PlanType {
+    SPLIT,
+    MERGE,
+    NONE
+  }
+
+  /**
+   * Executes normalization plan on cluster (does actual splitting/merging work).
+   * @param admin instance of Admin
+   */
+  void execute(Admin admin);
+
+  /**
+   * @return the type of this plan
+   */
+  PlanType getType();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
index c0083e6..0ee25f4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
@@ -25,8 +25,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 
 /**
  * Performs "normalization" of regions on the cluster, making sure that suboptimal
@@ -50,10 +49,9 @@ public interface RegionNormalizer {
   /**
    * Computes next optimal normalization plan.
    * @param table table to normalize
-   * @param types desired types of NormalizationPlan
    * @return normalization actions to perform. Null if no action to take
    */
-  List<NormalizationPlan> computePlanForTable(TableName table, List<PlanType> types)
+  List<NormalizationPlan> computePlanForTable(TableName table)
       throws HBaseIOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
index 7ea6cc7..bd522b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
@@ -27,8 +27,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -60,7 +59,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
   private static final Log LOG = LogFactory.getLog(SimpleRegionNormalizer.class);
   private static final int MIN_REGION_COUNT = 3;
   private MasterServices masterServices;
-  private static long[] skippedCount = new long[PlanType.values().length];
+  private static long[] skippedCount = new long[NormalizationPlan.PlanType.values().length];
 
   /**
    * Set the master service.
@@ -101,12 +100,10 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
    * Action may be either a split, or a merge, or no action.
    *
    * @param table table to normalize
-   * @param types desired types of NormalizationPlan
    * @return normalization plan to execute
    */
   @Override
-  public List<NormalizationPlan> computePlanForTable(TableName table, List<PlanType> types)
-      throws HBaseIOException {
+  public List<NormalizationPlan> computePlanForTable(TableName table) throws HBaseIOException {
     if (table == null || table.isSystemTable()) {
       LOG.debug("Normalization of system table " + table + " isn't allowed");
       return null;
@@ -146,7 +143,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
       long regionSize = getRegionSize(hri);
       // if the region is > 2 times larger than average, we split it, split
       // is more high priority normalization action than merge.
-      if (types.contains(PlanType.SPLIT) && regionSize > 2 * avgRegionSize) {
+      if (regionSize > 2 * avgRegionSize) {
         LOG.debug("Table " + table + ", large region " + hri.getRegionNameAsString() + " has size "
             + regionSize + ", more than twice avg size, splitting");
         plans.add(new SplitNormalizationPlan(hri, null));
@@ -156,7 +153,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
         }
         HRegionInfo hri2 = tableRegions.get(candidateIdx+1);
         long regionSize2 = getRegionSize(hri2);
-        if (types.contains(PlanType.MERGE) && regionSize + regionSize2 < avgRegionSize) {
+        if (regionSize + regionSize2 < avgRegionSize) {
           LOG.debug("Table " + table + ", small region size: " + regionSize
             + " plus its neighbor size: " + regionSize2
             + ", less than the avg size " + avgRegionSize + ", merging them");

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
index 76b7cc2..b95bfb7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
@@ -23,7 +23,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 
 import java.io.IOException;
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
index 1f66044..926c816 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
@@ -26,8 +26,6 @@ import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -55,18 +53,6 @@ public class TestSimpleRegionNormalizer {
   private static final Log LOG = LogFactory.getLog(TestSimpleRegionNormalizer.class);
 
   private static RegionNormalizer normalizer;
-  private static List<PlanType> bothTypes;
-  static {
-    bothTypes = new ArrayList<>();
-    bothTypes.add(PlanType.SPLIT);
-    bothTypes.add(PlanType.MERGE);
-  }
-
-  private static List<PlanType> splitType;
-  static {
-    splitType = new ArrayList<>();
-    splitType.add(PlanType.SPLIT);
-  }
 
   // mocks
   private static MasterServices masterServices;
@@ -83,7 +69,7 @@ public class TestSimpleRegionNormalizer {
     Map<byte[], Integer> regionSizes = new HashMap<>();
 
     setupMocksForNormalizer(regionSizes, hris);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable, bothTypes);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable);
     assertTrue(plans == null);
   }
 
@@ -102,7 +88,7 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri2.getRegionName(), 15);
 
     setupMocksForNormalizer(regionSizes, hris);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable, bothTypes);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable);
     assertTrue(plans == null);
   }
 
@@ -129,17 +115,12 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri4.getRegionName(), 10);
 
     setupMocksForNormalizer(regionSizes, hris);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable, bothTypes);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable);
     assertTrue(plans == null);
   }
 
   @Test
   public void testMergeOfSmallRegions() throws HBaseIOException {
-    testMergeOfSmallRegions(true);
-    testMergeOfSmallRegions(false);
-  }
-
-  public void testMergeOfSmallRegions(boolean mergeDesired) throws HBaseIOException {
     TableName testTable = TableName.valueOf("testMergeOfSmallRegions");
     List<HRegionInfo> hris = new ArrayList<>();
     Map<byte[], Integer> regionSizes = new HashMap<>();
@@ -165,17 +146,12 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri5.getRegionName(), 16);
 
     setupMocksForNormalizer(regionSizes, hris);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable,
-      mergeDesired ? bothTypes : splitType);
-
-    if (mergeDesired) {
-      NormalizationPlan plan = plans.get(0);
-      assertTrue(plan instanceof MergeNormalizationPlan);
-      assertEquals(hri2, ((MergeNormalizationPlan) plan).getFirstRegion());
-      assertEquals(hri3, ((MergeNormalizationPlan) plan).getSecondRegion());
-    } else {
-      assertTrue(plans == null);
-    }
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable);
+
+    NormalizationPlan plan = plans.get(0);
+    assertTrue(plan instanceof MergeNormalizationPlan);
+    assertEquals(hri2, ((MergeNormalizationPlan) plan).getFirstRegion());
+    assertEquals(hri3, ((MergeNormalizationPlan) plan).getSecondRegion());
   }
 
   // Test for situation illustrated in HBASE-14867
@@ -210,7 +186,7 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri6.getRegionName(), 2700);
 
     setupMocksForNormalizer(regionSizes, hris);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable, bothTypes);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable);
     NormalizationPlan plan = plans.get(0);
 
     assertTrue(plan instanceof MergeNormalizationPlan);
@@ -245,7 +221,7 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri5.getRegionName(), 5);
 
     setupMocksForNormalizer(regionSizes, hris);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable, bothTypes);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable);
 
     assertTrue(plans == null);
   }
@@ -273,7 +249,7 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri4.getRegionName(), 30);
 
     setupMocksForNormalizer(regionSizes, hris);
-    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable, bothTypes);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable);
     NormalizationPlan plan = plans.get(0);
 
     assertTrue(plan instanceof SplitNormalizationPlan);

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index 801e92d..4bcccc6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.TableNamespaceManager;
-import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.namespace.TestNamespaceAuditor;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -137,7 +137,7 @@ public class TestSimpleRegionNormalizerOnCluster {
     }
 
     HTableDescriptor htd = admin.getTableDescriptor(TABLENAME);
-    htd.setNormalizationMode("MS");
+    htd.setNormalizationEnabled(true);
     admin.modifyTable(TABLENAME, htd);
 
     admin.flush(TABLENAME);
@@ -215,7 +215,7 @@ public class TestSimpleRegionNormalizerOnCluster {
     }
 
     HTableDescriptor htd = admin.getTableDescriptor(TABLENAME);
-    htd.setNormalizationMode("MS");
+    htd.setNormalizationEnabled(true);
     admin.modifyTable(TABLENAME, htd);
 
     admin.flush(TABLENAME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 661783f..c61b598 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -280,10 +280,7 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Parse arguments and update HTableDescriptor accordingly
     def parse_htd_args(htd, arg)
-      if arg.has_key?(NORMALIZATION_MODE)
-        mode = arg.delete(NORMALIZATION_MODE)
-        htd.setValue(NORMALIZATION_MODE, mode)
-      end
+      htd.setNormalizationEnabled(JBoolean.valueOf(arg.delete(NORMALIZATION_ENABLED))) if arg[NORMALIZATION_ENABLED]
     end
 
     #----------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-shell/src/main/ruby/shell/commands/normalize.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/normalize.rb b/hbase-shell/src/main/ruby/shell/commands/normalize.rb
index e2b3d42..7e6302c 100644
--- a/hbase-shell/src/main/ruby/shell/commands/normalize.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/normalize.rb
@@ -22,7 +22,7 @@ module Shell
     class Normalize < Command
       def help
         return <<-EOF
-Trigger region normalizer for all tables which have NORMALIZATION_MODE flag set. Returns true
+Trigger region normalizer for all tables which have NORMALIZATION_ENABLED flag set. Returns true
  if normalizer ran successfully, false otherwise. Note that this command has no effect
  if region normalizer is disabled (make sure it's turned on using 'normalizer_switch' command).
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17f74b/hbase-shell/src/main/ruby/shell/commands/normalizer_switch.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/normalizer_switch.rb b/hbase-shell/src/main/ruby/shell/commands/normalizer_switch.rb
index ee9e2d1..6d959c4 100644
--- a/hbase-shell/src/main/ruby/shell/commands/normalizer_switch.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/normalizer_switch.rb
@@ -23,8 +23,7 @@ module Shell
       def help
         return <<-EOF
 Enable/Disable region normalizer. Returns previous normalizer state.
-When normalizer is enabled, it handles all tables with 'NORMALIZATION_MODE' flag containing
-types of normalization actions.
+When normalizer is enabled, it handles all tables with 'NORMALIZATION_ENABLED' => true.
 Examples:
 
   hbase> normalizer_switch true