You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2015/08/04 00:32:41 UTC

hbase git commit: HBASE-13965 Revert due to test failure in TestAssignmentManager

Repository: hbase
Updated Branches:
  refs/heads/branch-1 c215b900f -> 24dbe25e9


HBASE-13965 Revert due to test failure in TestAssignmentManager


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

Branch: refs/heads/branch-1
Commit: 24dbe25e95d0a355b2e07aa94b5921ff4b4865e9
Parents: c215b90
Author: tedyu <yu...@gmail.com>
Authored: Mon Aug 3 15:32:43 2015 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Aug 3 15:32:43 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/HConstants.java     |   6 -
 .../MetricsStochasticBalancerSource.java        |  39 ----
 .../MetricsStochasticBalancerSourceImpl.java    | 110 ---------
 ...ter.balancer.MetricsStochasticBalancerSource |  18 --
 .../org/apache/hadoop/hbase/master/HMaster.java |   7 +-
 .../hadoop/hbase/master/LoadBalancer.java       |  10 -
 .../hadoop/hbase/master/RegionStates.java       |   5 +-
 .../hbase/master/balancer/BaseLoadBalancer.java |  18 +-
 .../balancer/FavoredNodeLoadBalancer.java       |   8 -
 .../hbase/master/balancer/MetricsBalancer.java  |  10 +-
 .../balancer/MetricsStochasticBalancer.java     |  71 ------
 .../master/balancer/SimpleLoadBalancer.java     |   8 -
 .../master/balancer/StochasticLoadBalancer.java | 137 +----------
 .../hbase/TestStochasticBalancerJmxMetrics.java | 229 -------------------
 .../master/balancer/TestBaseLoadBalancer.java   |   7 -
 15 files changed, 14 insertions(+), 669 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 976bfc5..eb00a61 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -124,12 +124,6 @@ public final class HConstants {
   /** Config for pluggable load balancers */
   public static final String HBASE_MASTER_LOADBALANCER_CLASS = "hbase.master.loadbalancer.class";
 
-  /** Config for balancing the cluster by table */
-  public static final String HBASE_MASTER_LOADBALANCE_BYTABLE = "hbase.master.loadbalance.bytable";
-
-  /** The name of the ensemble table */
-  public static final String ENSEMBLE_TABLE_NAME = "hbase:ensemble";
-
   /** Config for pluggable region normalizer */
   public static final String HBASE_MASTER_NORMALIZER_CLASS =
     "hbase.master.normalizer.class";

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSource.java
deleted file mode 100644
index 1621932..0000000
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSource.java
+++ /dev/null
@@ -1,39 +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.master.balancer;
-
-/**
- * This interface extends the basic metrics balancer source to add a function 
- * to report metrics that related to stochastic load balancer. The purpose is to 
- * offer an insight to the internal cost calculations that can be useful to tune
- * the balancer. For details, refer to HBASE-13965
- */
-public interface MetricsStochasticBalancerSource extends MetricsBalancerSource {
-
-  /**
-   * Updates the number of metrics reported to JMX
-   */
-  public void updateMetricsSize(int size);
-
-  /**
-   * Reports stochastic load balancer costs to JMX
-   */
-  public void updateStochasticCost(String tableName, String costFunctionName,
-      String costFunctionDesc, Double value);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
deleted file mode 100644
index ded0a0c..0000000
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancerSourceImpl.java
+++ /dev/null
@@ -1,110 +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.master.balancer;
-
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.metrics2.MetricsCollector;
-import org.apache.hadoop.metrics2.MetricsRecordBuilder;
-import org.apache.hadoop.metrics2.lib.Interns;
-
-@InterfaceAudience.Private
-public class MetricsStochasticBalancerSourceImpl extends MetricsBalancerSourceImpl implements
-    MetricsStochasticBalancerSource {
-  private static final String TABLE_FUNCTION_SEP = "_";
-
-  // Most Recently Used(MRU) cache
-  private static final float MRU_LOAD_FACTOR = 0.75f;
-  private int metricsSize = 1000;
-  private int mruCap = calcMruCap(metricsSize);
-
-  private Map<String, Map<String, Double>> stochasticCosts =
-      new LinkedHashMap<String, Map<String, Double>>(mruCap, MRU_LOAD_FACTOR, true) {
-        private static final long serialVersionUID = 8204713453436906599L;
-
-        @Override
-        protected boolean removeEldestEntry(Map.Entry<String, Map<String, Double>> eldest) {
-          return size() > mruCap;
-        }
-      };
-  private Map<String, String> costFunctionDescs = new ConcurrentHashMap<String, String>();
-
-  /**
-   * Calculates the mru cache capacity from the metrics size
-   */
-  private static int calcMruCap(int metricsSize) {
-    return (int) Math.ceil(metricsSize / MRU_LOAD_FACTOR) + 1;
-  }
-
-  @Override
-  public void updateMetricsSize(int size) {
-    if (size > 0) {
-      metricsSize = size;
-      mruCap = calcMruCap(size);
-    }
-  }
-
-  /**
-   * Reports stochastic load balancer costs to JMX
-   */
-  public void updateStochasticCost(String tableName, String costFunctionName, String functionDesc,
-      Double cost) {
-    if (tableName == null || costFunctionName == null || cost == null) {
-      return;
-    }
-
-    if (functionDesc != null) {
-      costFunctionDescs.put(costFunctionName, functionDesc);
-    }
-
-    synchronized (stochasticCosts) {
-      Map<String, Double> costs = stochasticCosts.get(tableName);
-      if (costs == null) {
-        costs = new ConcurrentHashMap<String, Double>();
-      }
-
-      costs.put(costFunctionName, cost);
-      stochasticCosts.put(tableName, costs);
-    }
-  }
-
-  @Override
-  public void getMetrics(MetricsCollector metricsCollector, boolean all) {
-    MetricsRecordBuilder metricsRecordBuilder = metricsCollector.addRecord(metricsName);
-
-    if (stochasticCosts != null) {
-      synchronized (stochasticCosts) {
-        for (Map.Entry<String, Map<String, Double>> tableEntry : stochasticCosts.entrySet()) {
-          for (Map.Entry<String, Double> costEntry : tableEntry.getValue().entrySet()) {
-            String attrName = tableEntry.getKey() + TABLE_FUNCTION_SEP + costEntry.getKey();
-            Double cost = costEntry.getValue();
-            String functionDesc = costFunctionDescs.get(costEntry.getKey());
-            if (functionDesc == null) functionDesc = costEntry.getKey();
-            metricsRecordBuilder.addGauge(Interns.info(attrName, functionDesc), cost);
-          }
-        }
-      }
-    }
-    metricsRegistry.snapshot(metricsRecordBuilder, all);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsStochasticBalancerSource
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsStochasticBalancerSource b/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsStochasticBalancerSource
deleted file mode 100644
index 80c0895..0000000
--- a/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsStochasticBalancerSource
+++ /dev/null
@@ -1,18 +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.
-#
-org.apache.hadoop.hbase.master.balancer.MetricsStochasticBalancerSourceImpl
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/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 b57b993..423deaf 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
@@ -33,7 +33,6 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
@@ -1254,14 +1253,12 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
         this.assignmentManager.getRegionStates().getAssignmentsByTable();
 
       List<RegionPlan> plans = new ArrayList<RegionPlan>();
-
       //Give the balancer the current cluster state.
       this.balancer.setClusterStatus(getClusterStatus());
-      for (Entry<TableName, Map<ServerName, List<HRegionInfo>>> e : assignmentsByTable.entrySet()) {
-        List<RegionPlan> partialPlans = this.balancer.balanceCluster(e.getKey(), e.getValue());
+      for (Map<ServerName, List<HRegionInfo>> assignments : assignmentsByTable.values()) {
+        List<RegionPlan> partialPlans = this.balancer.balanceCluster(assignments);
         if (partialPlans != null) plans.addAll(partialPlans);
       }
-
       long cutoffTime = System.currentTimeMillis() + maximumBalanceTime;
       int rpCount = 0;  // number of RegionPlans balanced so far
       long totalRegPlanExecTime = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
index e293d60..f979403 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.TableName;
 
 /**
  * Makes decisions about the placement and movement of Regions across
@@ -66,15 +65,6 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
 
   /**
    * Perform the major balance operation
-   * @param tableName
-   * @param clusterState
-   * @return List of plans
-   */
-  List<RegionPlan> balanceCluster(TableName tableName, Map<ServerName,
-      List<HRegionInfo>> clusterState) throws HBaseIOException;
-
-  /**
-   * Perform the major balance operation
    * @param clusterState
    * @return List of plans
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
index 39ddab0..c658475 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
@@ -991,14 +991,13 @@ public class RegionStates {
     Map<TableName, Map<ServerName, List<HRegionInfo>>> result =
       new HashMap<TableName, Map<ServerName,List<HRegionInfo>>>();
     synchronized (this) {
-      if (!server.getConfiguration().getBoolean(
-            HConstants.HBASE_MASTER_LOADBALANCE_BYTABLE, false)) {
+      if (!server.getConfiguration().getBoolean("hbase.master.loadbalance.bytable", false)) {
         Map<ServerName, List<HRegionInfo>> svrToRegions =
           new HashMap<ServerName, List<HRegionInfo>>(serverHoldings.size());
         for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
           svrToRegions.put(e.getKey(), new ArrayList<HRegionInfo>(e.getValue()));
         }
-        result.put(TableName.valueOf(HConstants.ENSEMBLE_TABLE_NAME), svrToRegions);
+        result.put(TableName.valueOf("ensemble"), svrToRegions);
       } else {
         for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
           for (HRegionInfo hri: e.getValue()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 3843745..97afa49 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RackManager;
@@ -77,21 +78,6 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       return UNKNOWN_RACK;
     }
   }
-  
-  /**
-   * The constructor that uses the basic MetricsBalancer
-   */
-  protected BaseLoadBalancer() {
-    metricsBalancer = new MetricsBalancer();
-  }
-  
-  /**
-   * This Constructor accepts an instance of MetricsBalancer, 
-   * which will be used instead of creating a new one
-   */
-  protected BaseLoadBalancer(MetricsBalancer metricsBalancer) {
-    this.metricsBalancer = (metricsBalancer != null) ? metricsBalancer : new MetricsBalancer();
-  }
 
   /**
    * An efficient array based implementation similar to ClusterState for keeping
@@ -809,7 +795,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
     "hbase.balancer.tablesOnMaster";
 
   protected final Set<String> tablesOnMaster = new HashSet<String>();
-  protected MetricsBalancer metricsBalancer = null;
+  protected final MetricsBalancer metricsBalancer = new MetricsBalancer();
   protected ClusterStatus clusterStatus = null;
   protected ServerName masterServerName;
   protected MasterServices services;

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java
index 32d9c6c..3560447 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java
@@ -28,13 +28,11 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.RackManager;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.ServerManager;
@@ -346,10 +344,4 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer {
       globalFavoredNodesAssignmentPlan.updateFavoredNodesMap(region, favoredNodesForRegion);
     }
   }
-
-  @Override
-  public List<RegionPlan> balanceCluster(TableName tableName,
-      Map<ServerName, List<HRegionInfo>> clusterState) throws HBaseIOException {
-    return balanceCluster(clusterState);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancer.java
index 3707536..518c2f0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancer.java
@@ -25,17 +25,9 @@ import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
  */
 public class MetricsBalancer {
 
-  private MetricsBalancerSource source = null;
+  private final MetricsBalancerSource source;
 
   public MetricsBalancer() {
-    initSource();
-  }
-  
-  /**
-   * A function to instantiate the metrics source. This function can be overridden in its 
-   * subclasses to provide extended sources
-   */
-  protected void initSource() {
     source = CompatibilitySingletonFactory.getInstance(MetricsBalancerSource.class);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancer.java
deleted file mode 100644
index 850a9f5..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancer.java
+++ /dev/null
@@ -1,71 +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.master.balancer;
-
-import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
-
-/**
- * This metrics balancer uses extended source for stochastic load balancer
- * to report its related metrics to JMX. For details, refer to HBASE-13965
- */
-public class MetricsStochasticBalancer extends MetricsBalancer {
-  /**
-   * Use the stochastic source instead of the default source.
-   */
-  private MetricsStochasticBalancerSource stochasticSource = null;
-
-  public MetricsStochasticBalancer() {
-    initSource();
-  }
-
-  /**
-   * This function overrides the initSource in the MetricsBalancer, use
-   * MetricsStochasticBalancerSource instead of the MetricsBalancerSource.
-   */
-  @Override
-  protected void initSource() {
-    stochasticSource =
-        CompatibilitySingletonFactory.getInstance(MetricsStochasticBalancerSource.class);
-  }
-
-  @Override
-  public void balanceCluster(long time) {
-    stochasticSource.updateBalanceCluster(time);
-  }
-
-  @Override
-  public void incrMiscInvocations() {
-    stochasticSource.incrMiscInvocations();
-  }
-
-  /**
-   * Updates the number of metrics reported to JMX
-   */
-  public void updateMetricsSize(int size) {
-    stochasticSource.updateMetricsSize(size);
-  }
-
-  /**
-   * Reports stochastic load balancer costs to JMX
-   */
-  public void updateStochasticCost(String tableName, String costFunctionName,
-      String costFunctionDesc, Double value) {
-    stochasticSource.updateStochasticCost(tableName, costFunctionName, costFunctionDesc, value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
index 4325585..fad84f5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
@@ -30,11 +30,9 @@ import java.util.TreeMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.RegionPlan;
 
 import com.google.common.collect.MinMaxPriorityQueue;
@@ -435,10 +433,4 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
     rp.setDestination(sn);
     regionsToReturn.add(rp);
   }
-
-  @Override
-  public List<RegionPlan> balanceCluster(TableName tableName,
-      Map<ServerName, List<HRegionInfo>> clusterState) throws HBaseIOException {
-    return balanceCluster(clusterState);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
index b76706f..b6b4691 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
@@ -34,13 +34,10 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action;
@@ -105,7 +102,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
       "hbase.master.balancer.stochastic.maxRunningTime";
   protected static final String KEEP_REGION_LOADS =
       "hbase.master.balancer.stochastic.numRegionLoadsToRemember";
-  private static final String TABLE_FUNCTION_SEP = "_";
 
   private static final Random RANDOM = new Random(System.currentTimeMillis());
   private static final Log LOG = LogFactory.getLog(StochasticLoadBalancer.class);
@@ -121,28 +117,12 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
   private CandidateGenerator[] candidateGenerators;
   private CostFromRegionLoadFunction[] regionLoadFunctions;
   private CostFunction[] costFunctions;
-
-  // to save and report costs to JMX
-  private Double curOverallCost = 0d;
-  private Double[] tempFunctionCosts;
-  private Double[] curFunctionCosts;
-
   // Keep locality based picker and cost function to alert them
   // when new services are offered
   private LocalityBasedCandidateGenerator localityCandidateGenerator;
   private LocalityCostFunction localityCost;
   private RegionReplicaHostCostFunction regionReplicaHostCostFunction;
   private RegionReplicaRackCostFunction regionReplicaRackCostFunction;
-  private boolean isByTable = false;
-  private TableName tableName = null;
-  
-  /**
-   * The constructor that pass a MetricsStochasticBalancer to BaseLoadBalancer to replace its
-   * default MetricsBalancer
-   */
-  public StochasticLoadBalancer() {
-    super(new MetricsStochasticBalancer());
-  }
 
   @Override
   public void onConfigurationChange(Configuration conf) {
@@ -160,7 +140,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     maxRunningTime = conf.getLong(MAX_RUNNING_TIME_KEY, maxRunningTime);
 
     numRegionLoadsToRemember = conf.getInt(KEEP_REGION_LOADS, numRegionLoadsToRemember);
-    isByTable = conf.getBoolean(HConstants.HBASE_MASTER_LOADBALANCE_BYTABLE, isByTable);
 
     if (localityCandidateGenerator == null) {
       localityCandidateGenerator = new LocalityBasedCandidateGenerator(services);
@@ -199,10 +178,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
       regionLoadFunctions[2],
       regionLoadFunctions[3],
     };
-    
-    curFunctionCosts= new Double[costFunctions.length];
-    tempFunctionCosts= new Double[costFunctions.length];
-
   }
 
   @Override
@@ -217,26 +192,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     for(CostFromRegionLoadFunction cost : regionLoadFunctions) {
       cost.setClusterStatus(st);
     }
-
-    // update metrics size
-    try {
-      // by-table or ensemble mode
-      int tablesCount = isByTable ? services.getTableDescriptors().getAll().size() : 1;
-      int functionsCount = getCostFunctionNames().length;
-
-      updateMetricsSize(tablesCount * (functionsCount + 1)); // +1 for overall
-    } catch (Exception e) {
-      LOG.error("failed to get the size of all tables, exception = " + e.getMessage());
-    }
-  }
-  
-  /**
-   * Update the number of metrics that are reported to JMX
-   */
-  public void updateMetricsSize(int size) {
-    if (metricsBalancer instanceof MetricsStochasticBalancer) {
-        ((MetricsStochasticBalancer) metricsBalancer).updateMetricsSize(size);
-    }
   }
 
   @Override
@@ -256,13 +211,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     return false;
   }
 
-  @Override
-  public synchronized List<RegionPlan> balanceCluster(TableName tableName, Map<ServerName,
-    List<HRegionInfo>> clusterState) {
-    this.tableName = tableName;
-    return balanceCluster(clusterState);
-  }
-  
   /**
    * Given the cluster state this will try and approach an optimal balance. This
    * should always approach the optimal state given enough steps.
@@ -274,7 +222,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     if (plans != null || clusterState == null || clusterState.size() <= 1) {
       return plans;
     }
-
     if (masterServerName != null && clusterState.containsKey(masterServerName)) {
       if (clusterState.size() <= 2) {
         return null;
@@ -296,7 +243,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     //of all the regions in the table(s) (that's true today)
     // Keep track of servers to iterate through them.
     Cluster cluster = new Cluster(clusterState, loads, finder, rackManager);
-
     if (!needsBalance(cluster)) {
       return null;
     }
@@ -306,10 +252,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     initCosts(cluster);
 
     double currentCost = computeCost(cluster, Double.MAX_VALUE);
-    curOverallCost = currentCost;
-    for (int i = 0; i < this.curFunctionCosts.length; i++) {
-      curFunctionCosts[i] = tempFunctionCosts[i];
-    }
 
     double initCost = currentCost;
     double newCost = currentCost;
@@ -336,12 +278,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
       // Should this be kept?
       if (newCost < currentCost) {
         currentCost = newCost;
-
-        // save for JMX
-        curOverallCost = currentCost;
-        for (int i = 0; i < this.curFunctionCosts.length; i++) {
-          curFunctionCosts[i] = tempFunctionCosts[i];
-        }
       } else {
         // Put things back the way they were before.
         // TODO: undo by remembering old values
@@ -360,8 +296,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
 
     metricsBalancer.balanceCluster(endTime - startTime);
 
-    // update costs metrics
-    updateStochasticCosts(tableName, curOverallCost, curFunctionCosts);
     if (initCost > currentCost) {
       plans = createRegionPlans(cluster);
       if (LOG.isDebugEnabled()) {
@@ -371,7 +305,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
             + plans.size() + " regions; Going from a computed cost of "
             + initCost + " to a new cost of " + currentCost);
       }
-      
       return plans;
     }
     if (LOG.isDebugEnabled()) {
@@ -381,32 +314,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     }
     return null;
   }
-  
-  /**
-   * update costs to JMX
-   */
-  private void updateStochasticCosts(TableName tableName, Double overall, Double[] subCosts) {
-    if (tableName == null) return;
-
-    // check if the metricsBalancer is MetricsStochasticBalancer before casting
-    if (metricsBalancer instanceof MetricsStochasticBalancer) {
-      MetricsStochasticBalancer balancer = (MetricsStochasticBalancer) metricsBalancer;
-      // overall cost
-      balancer.updateStochasticCost(tableName.getNameAsString(), 
-        "Overall", "Overall cost", overall);
-
-      // each cost function
-      for (int i = 0; i < costFunctions.length; i++) {
-        CostFunction costFunction = costFunctions[i];
-        String costFunctionName = costFunction.getClass().getSimpleName();
-        Double costPercent = (overall == 0) ? 0 : (subCosts[i] / overall);
-        // TODO: cost function may need a specific description
-        balancer.updateStochasticCost(tableName.getNameAsString(), costFunctionName,
-          "The percent of " + costFunctionName, costPercent);
-      }
-    }
-  }
-
 
   /**
    * Create all of the RegionPlan's needed to move from the initial cluster state to the desired
@@ -484,20 +391,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
   }
 
   /**
-   * Get the names of the cost functions
-   */
-  public String[] getCostFunctionNames() {
-    if (costFunctions == null) return null;
-    String[] ret = new String[costFunctions.length];
-    for (int i = 0; i < costFunctions.length; i++) {
-      CostFunction c = costFunctions[i];
-      ret[i] = c.getClass().getSimpleName();
-    }
-
-    return ret;
-  }
-
-  /**
    * This is the main cost function.  It will compute a cost associated with a proposed cluster
    * state.  All different costs will be combined with their multipliers to produce a double cost.
    *
@@ -509,25 +402,17 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
   protected double computeCost(Cluster cluster, double previousCost) {
     double total = 0;
 
-    for (int i = 0; i < costFunctions.length; i++) {
-      CostFunction c = costFunctions[i];
-      this.tempFunctionCosts[i] = 0.0;
-      
+    for (CostFunction c:costFunctions) {
       if (c.getMultiplier() <= 0) {
         continue;
       }
 
-      Float multiplier = c.getMultiplier();
-      Double cost = c.cost();
-
-      this.tempFunctionCosts[i] = multiplier*cost;
-      total += this.tempFunctionCosts[i];
+      total += c.getMultiplier() * c.cost();
 
       if (total > previousCost) {
-        break;
+        return total;
       }
     }
-   
     return total;
   }
 
@@ -618,7 +503,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
       return getAction(thisServer, thisRegion, otherServer, otherRegion);
     }
 
-    protected Cluster.Action getAction(int fromServer, int fromRegion,
+    protected Cluster.Action getAction (int fromServer, int fromRegion,
         int toServer, int toRegion) {
       if (fromServer < 0 || toServer < 0) {
         return Cluster.NullAction;
@@ -826,7 +711,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
 
       int toServerIndex = pickOtherRandomServer(cluster, serverIndex);
       int toRegionIndex = pickRandomRegion(cluster, toServerIndex, 0.9f);
-      return getAction(serverIndex, regionIndex, toServerIndex, toRegionIndex);
+      return getAction (serverIndex, regionIndex, toServerIndex, toRegionIndex);
     }
   }
 
@@ -859,7 +744,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
       int rand = RANDOM.nextInt(cluster.serversPerRack[toRackIndex].length);
       int toServerIndex = cluster.serversPerRack[toRackIndex][rand];
       int toRegionIndex = pickRandomRegion(cluster, toServerIndex, 0.9f);
-      return getAction(serverIndex, regionIndex, toServerIndex, toRegionIndex);
+      return getAction (serverIndex, regionIndex, toServerIndex, toRegionIndex);
     }
   }
 
@@ -1433,8 +1318,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
 
     public RegionReplicaRackCostFunction(Configuration conf) {
       super(conf);
-      this.setMultiplier(conf.getFloat(REGION_REPLICA_RACK_COST_KEY, 
-        DEFAULT_REGION_REPLICA_RACK_COST_KEY));
+      this.setMultiplier(conf.getFloat(REGION_REPLICA_RACK_COST_KEY, DEFAULT_REGION_REPLICA_RACK_COST_KEY));
     }
 
     @Override
@@ -1506,11 +1390,4 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
       return rl.getStorefileSizeMB();
     }
   }
-  
-  /**
-   * A helper function to compose the attribute name from tablename and costfunction name
-   */
-  public static String composeAttributeName(String tableName, String costFunctionName) {
-    return tableName + TABLE_FUNCTION_SEP + costFunctionName;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
deleted file mode 100644
index d0ff589..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
+++ /dev/null
@@ -1,229 +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;
-
-import static org.junit.Assert.assertTrue;
-
-import java.util.HashSet;
-import java.util.Hashtable;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import javax.management.MBeanAttributeInfo;
-import javax.management.MBeanInfo;
-import javax.management.MBeanServerConnection;
-import javax.management.ObjectName;
-import javax.management.remote.JMXConnector;
-import javax.management.remote.JMXConnectorFactory;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.master.balancer.BalancerTestBase;
-import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.net.DNSToSwitchMapping;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.FixMethodOrder;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runners.MethodSorters;
-
-@Category({ MediumTests.class })
-@FixMethodOrder(MethodSorters.NAME_ASCENDING)
-public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
-  private static final Log LOG = LogFactory.getLog(TestStochasticBalancerJmxMetrics.class);
-  private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
-  private static int connectorPort = 61120;
-  private static StochasticLoadBalancer loadBalancer;
-  /**
-   * a simple cluster for testing JMX.
-   */
-  private static int[] mockCluster_ensemble = new int[] { 0, 1, 2, 3 };
-  private static int[] mockCluster_pertable_1 = new int[] { 0, 1, 2 };
-  private static int[] mockCluster_pertable_2 = new int[] { 3, 1, 1 };
-  private static int[] mockCluster_pertable_namespace = new int[] { 1, 3, 1 };
-
-  private static final String TABLE_NAME_1 = "Table1";
-  private static final String TABLE_NAME_2 = "Table2";
-  private static final String TABLE_NAME_NAMESPACE = "hbase:namespace";
-
-  private static Configuration conf = null;
-
-  /**
-   * Setup the environment for the test.
-   */
-  @BeforeClass
-  public static void setupBeforeClass() throws Exception {
-
-    conf = UTIL.getConfiguration();
-
-    conf.setClass("hbase.util.ip.to.rack.determiner", MockMapping.class, DNSToSwitchMapping.class);
-    conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 0.75f);
-    conf.setFloat("hbase.regions.slop", 0.0f);
-    conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, JMXListener.class.getName());
-    conf.setInt("regionserver.rmi.registry.port", connectorPort);
-
-    UTIL.startMiniCluster();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    UTIL.shutdownMiniCluster();
-  }
-
-  /**
-   * In Ensemble mode, there should be only one ensemble table
-   */
-  @Test
-  public void testJmxMetrics_EnsembleMode() throws Exception {
-    loadBalancer = new StochasticLoadBalancer();
-
-    conf.setBoolean(HConstants.HBASE_MASTER_LOADBALANCE_BYTABLE, false);
-    loadBalancer.setConf(conf);
-
-    TableName tableName = TableName.valueOf(HConstants.ENSEMBLE_TABLE_NAME);
-    Map<ServerName, List<HRegionInfo>> clusterState = mockClusterServers(mockCluster_ensemble);
-    loadBalancer.balanceCluster(tableName, clusterState);
-
-    String[] tableNames = new String[] { tableName.getNameAsString() };
-    String[] functionNames = loadBalancer.getCostFunctionNames();
-    Set<String> jmxMetrics = readJmxMetrics();
-    Set<String> expectedMetrics = getExpectedJmxMetrics(tableNames, functionNames);
-
-    // printMetrics(jmxMetrics, "existing metrics in ensemble mode");
-    // printMetrics(expectedMetrics, "expected metrics in ensemble mode");
-
-    // assert that every expected is in the JMX
-    for (String expected : expectedMetrics) {
-      assertTrue("Metric " + expected + " can not be found in JMX in ensemble mode.",
-        jmxMetrics.contains(expected));
-    }
-  }
-
-  /**
-   * In per-table mode, each table has a set of metrics
-   */
-  @Test
-  public void testJmxMetrics_PerTableMode() throws Exception {
-    loadBalancer = new StochasticLoadBalancer();
-
-    conf.setBoolean(HConstants.HBASE_MASTER_LOADBALANCE_BYTABLE, true);
-    loadBalancer.setConf(conf);
-
-    // NOTE the size is normally set in setClusterStatus, for test purpose, we set it manually
-    // Tables: hbase:namespace, table1, table2
-    // Functions: costFunctions, overall
-    String[] functionNames = loadBalancer.getCostFunctionNames();
-    loadBalancer.updateMetricsSize(3 * (functionNames.length + 1));
-
-    // table 1
-    TableName tableName = TableName.valueOf(TABLE_NAME_1);
-    Map<ServerName, List<HRegionInfo>> clusterState = mockClusterServers(mockCluster_pertable_1);
-    loadBalancer.balanceCluster(tableName, clusterState);
-
-    // table 2
-    tableName = TableName.valueOf(TABLE_NAME_2);
-    clusterState = mockClusterServers(mockCluster_pertable_2);
-    loadBalancer.balanceCluster(tableName, clusterState);
-
-    // table hbase:namespace
-    tableName = TableName.valueOf(TABLE_NAME_NAMESPACE);
-    clusterState = mockClusterServers(mockCluster_pertable_namespace);
-    loadBalancer.balanceCluster(tableName, clusterState);
-
-    String[] tableNames = new String[] { TABLE_NAME_1, TABLE_NAME_2, TABLE_NAME_NAMESPACE };
-    Set<String> jmxMetrics = readJmxMetrics();
-    Set<String> expectedMetrics = getExpectedJmxMetrics(tableNames, functionNames);
-
-    // printMetrics(jmxMetrics, "existing metrics in per-table mode");
-    // printMetrics(expectedMetrics, "expected metrics in per-table mode");
-
-    // assert that every expected is in the JMX
-    for (String expected : expectedMetrics) {
-      assertTrue("Metric " + expected + " can not be found in JMX in per-table mode.",
-        jmxMetrics.contains(expected));
-    }
-  }
-
-  /**
-   * Read the attributes from Hadoop->HBase->Master->Balancer in JMX
-   */
-  private Set<String> readJmxMetrics() {
-    JMXConnector connector = null;
-    try {
-      connector =
-          JMXConnectorFactory.connect(JMXListener.buildJMXServiceURL(connectorPort, connectorPort));
-      MBeanServerConnection mb = connector.getMBeanServerConnection();
-
-      Hashtable<String, String> pairs = new Hashtable<>();
-      pairs.put("service", "HBase");
-      pairs.put("name", "Master");
-      pairs.put("sub", "Balancer");
-      ObjectName target = new ObjectName("Hadoop", pairs);
-      MBeanInfo beanInfo = mb.getMBeanInfo(target);
-
-      Set<String> existingAttrs = new HashSet<String>();
-      for (MBeanAttributeInfo attrInfo : beanInfo.getAttributes()) {
-        existingAttrs.add(attrInfo.getName());
-      }
-      return existingAttrs;
-    } catch (Exception e) {
-      e.printStackTrace();
-    } finally {
-      if (connector != null) {
-        try {
-          connector.close();
-        } catch (Exception e) {
-          e.printStackTrace();
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Given the tables and functions, return metrics names that should exist in JMX
-   */
-  private Set<String> getExpectedJmxMetrics(String[] tableNames, String[] functionNames) {
-    Set<String> ret = new HashSet<String>();
-
-    for (String tableName : tableNames) {
-      ret.add(StochasticLoadBalancer.composeAttributeName(tableName, "Overall"));
-      for (String functionName : functionNames) {
-        String metricsName = StochasticLoadBalancer.composeAttributeName(tableName, functionName);
-        ret.add(metricsName);
-      }
-    }
-
-    return ret;
-  }
-
-  private static void printMetrics(Set<String> metrics, String info) {
-    if (null != info) LOG.info("++++ ------ " + info + " ------");
-
-    LOG.info("++++ metrics count = " + metrics.size());
-    for (String str : metrics) {
-      LOG.info(" ++++ " + str);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/24dbe25e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
index be63d91..3bdae33 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
@@ -36,7 +36,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.ServerName;
@@ -106,12 +105,6 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
       return null;
     }
 
-    @Override
-    public List<RegionPlan> balanceCluster(TableName tableName,
-        Map<ServerName, List<HRegionInfo>> clusterState) throws HBaseIOException {
-      return null;
-    }
-
   }
 
   /**