You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ch...@apache.org on 2018/01/04 05:16:51 UTC

[1/3] hbase git commit: HBASE-19596 RegionMetrics/ServerMetrics/ClusterMetrics should apply to all public classes

Repository: hbase
Updated Branches:
  refs/heads/branch-2 72631a08c -> 654edc5fa


http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
index 58ae059..59a0059 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
@@ -19,16 +19,16 @@ package org.apache.hadoop.hbase;
 
 import java.io.Closeable;
 import java.io.IOException;
-
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
-import org.apache.hadoop.hbase.util.Threads;
 
 /**
  * This class defines methods that can help with managing HBase clusters
@@ -61,7 +61,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
   protected Configuration conf;
 
   /** the status of the cluster before we begin */
-  protected ClusterStatus initialClusterStatus;
+  protected ClusterMetrics initialClusterStatus;
 
   /**
    * Construct an HBaseCluster
@@ -82,16 +82,16 @@ public abstract class HBaseCluster implements Closeable, Configurable {
   }
 
   /**
-   * Returns a ClusterStatus for this HBase cluster.
-   * @see #getInitialClusterStatus()
+   * Returns a ClusterMetrics for this HBase cluster.
+   * @see #getInitialClusterMetrics()
    */
-  public abstract ClusterStatus getClusterStatus() throws IOException;
+  public abstract ClusterMetrics getClusterMetrics() throws IOException;
 
   /**
    * Returns a ClusterStatus for this HBase cluster as observed at the
    * starting of the HBaseCluster
    */
-  public ClusterStatus getInitialClusterStatus() throws IOException {
+  public ClusterMetrics getInitialClusterMetrics() throws IOException {
     return initialClusterStatus;
   }
 
@@ -153,7 +153,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
       throws IOException {
     long start = System.currentTimeMillis();
     while ((System.currentTimeMillis() - start) < timeout) {
-      for (ServerName server : getClusterStatus().getServers()) {
+      for (ServerName server : getClusterMetrics().getLiveServerMetrics().keySet()) {
         if (server.getHostname().equals(hostname) && server.getPort() == port) {
           return;
         }
@@ -317,7 +317,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
    * @return whether restoration is complete
    */
   public boolean restoreInitialStatus() throws IOException {
-    return restoreClusterStatus(getInitialClusterStatus());
+    return restoreClusterMetrics(getInitialClusterMetrics());
   }
 
   /**
@@ -327,7 +327,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
    * permissions, etc. restoration might be partial.
    * @return whether restoration is complete
    */
-  public boolean restoreClusterStatus(ClusterStatus desiredStatus) throws IOException {
+  public boolean restoreClusterMetrics(ClusterMetrics desiredStatus) throws IOException {
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 9e17a79..304b3cb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import edu.umd.cs.findbugs.annotations.Nullable;
-
 import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
@@ -52,7 +51,6 @@ import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
-
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.logging.impl.Jdk14Logger;
@@ -151,6 +149,7 @@ import org.apache.zookeeper.ZooKeeper.States;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.impl.Log4jLoggerAdapter;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 
 /**
@@ -3735,7 +3734,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       // The number of splits is set as:
       //    region servers * regions per region server).
       int numberOfServers =
-          admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()
+          admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics()
               .size();
       if (numberOfServers == 0) {
         throw new IllegalStateException("No live regionservers");

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
index ad58124..92715ee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
@@ -23,19 +23,10 @@ import java.security.PrivilegedAction;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
-
 import java.util.Set;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
@@ -45,6 +36,14 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
 
 /**
  * This class creates a single process HBase cluster.
@@ -629,10 +628,20 @@ public class MiniHBaseCluster extends HBaseCluster {
   public void close() throws IOException {
   }
 
-  @Override
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getClusterMetrics()} instead.
+   */
+  @Deprecated
   public ClusterStatus getClusterStatus() throws IOException {
     HMaster master = getMaster();
-    return master == null ? null : master.getClusterStatus();
+    return master == null ? null : new ClusterStatus(master.getClusterMetrics());
+  }
+
+  @Override
+  public ClusterMetrics getClusterMetrics() throws IOException {
+    HMaster master = getMaster();
+    return master == null ? null : master.getClusterMetrics();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
index ac116d8..b268842 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
@@ -23,7 +23,6 @@ import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.Waiter.Predicate;
@@ -77,8 +76,8 @@ public class TestClientClusterMetrics {
 
   @Test
   public void testDefaults() throws Exception {
-    ClusterMetrics origin = ADMIN.getClusterStatus();
-    ClusterMetrics defaults = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
+    ClusterMetrics origin = ADMIN.getClusterMetrics();
+    ClusterMetrics defaults = ADMIN.getClusterMetrics(EnumSet.allOf(Option.class));
     Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
     Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
     Assert.assertEquals(origin.getAverageLoad(), defaults.getAverageLoad(), 0);
@@ -96,10 +95,10 @@ public class TestClientClusterMetrics {
     try (AsyncConnection asyncConnect = ConnectionFactory.createAsyncConnection(
       UTIL.getConfiguration()).get()) {
       AsyncAdmin asyncAdmin = asyncConnect.getAdmin();
-      CompletableFuture<ClusterStatus> originFuture =
-        asyncAdmin.getClusterStatus();
-      CompletableFuture<ClusterStatus> defaultsFuture =
-        asyncAdmin.getClusterStatus(EnumSet.allOf(Option.class));
+      CompletableFuture<ClusterMetrics> originFuture =
+        asyncAdmin.getClusterMetrics();
+      CompletableFuture<ClusterMetrics> defaultsFuture =
+        asyncAdmin.getClusterMetrics(EnumSet.allOf(Option.class));
       ClusterMetrics origin = originFuture.get();
       ClusterMetrics defaults = defaultsFuture.get();
       Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
@@ -133,14 +132,14 @@ public class TestClientClusterMetrics {
     Waiter.waitFor(CLUSTER.getConfiguration(), 10 * 1000, 100, new Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        ClusterMetrics metrics = ADMIN.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+        ClusterMetrics metrics = ADMIN.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS));
         Assert.assertNotNull(metrics);
         return metrics.getRegionCount() > 0;
       }
     });
     // Retrieve live servers and dead servers info.
     EnumSet<Option> options = EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS);
-    ClusterMetrics metrics = ADMIN.getClusterStatus(options);
+    ClusterMetrics metrics = ADMIN.getClusterMetrics(options);
     Assert.assertNotNull(metrics);
     // exclude a dead region server
     Assert.assertEquals(SLAVES -1, numRs);
@@ -176,7 +175,7 @@ public class TestClientClusterMetrics {
     Assert.assertEquals(MASTERS, masterThreads.size());
     // Retrieve master and backup masters infos only.
     EnumSet<Option> options = EnumSet.of(Option.MASTER, Option.BACKUP_MASTERS);
-    ClusterMetrics metrics = ADMIN.getClusterStatus(options);
+    ClusterMetrics metrics = ADMIN.getClusterMetrics(options);
     Assert.assertTrue(metrics.getMasterName().equals(activeName));
     Assert.assertEquals(MASTERS - 1, metrics.getBackupMasterNames().size());
   }
@@ -186,7 +185,7 @@ public class TestClientClusterMetrics {
     EnumSet<Option> options =
         EnumSet.of(Option.MASTER_COPROCESSORS, Option.HBASE_VERSION,
                    Option.CLUSTER_ID, Option.BALANCER_ON);
-    ClusterMetrics metrics = ADMIN.getClusterStatus(options);
+    ClusterMetrics metrics = ADMIN.getClusterMetrics(options);
     Assert.assertEquals(1, metrics.getMasterCoprocessorNames().size());
     Assert.assertNotNull(metrics.getHBaseVersion());
     Assert.assertNotNull(metrics.getClusterId());
@@ -206,7 +205,7 @@ public class TestClientClusterMetrics {
   public void testObserver() throws IOException {
     int preCount = MyObserver.PRE_COUNT.get();
     int postCount = MyObserver.POST_COUNT.get();
-    Assert.assertTrue(Stream.of(ADMIN.getClusterStatus().getMasterCoprocessors())
+    Assert.assertTrue(ADMIN.getClusterMetrics().getMasterCoprocessorNames().stream()
         .anyMatch(s -> s.equals(MyObserver.class.getSimpleName())));
     Assert.assertEquals(preCount + 1, MyObserver.PRE_COUNT.get());
     Assert.assertEquals(postCount + 1, MyObserver.POST_COUNT.get());
@@ -220,13 +219,13 @@ public class TestClientClusterMetrics {
       return Optional.of(this);
     }
 
-    @Override public void preGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx)
+    @Override public void preGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx)
         throws IOException {
       PRE_COUNT.incrementAndGet();
     }
 
-    @Override public void postGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        ClusterStatus metrics) throws IOException {
+    @Override public void postGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        ClusterMetrics metrics) throws IOException {
       POST_COUNT.incrementAndGet();
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
index 245f3c9..d7e6f14 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
@@ -21,16 +21,12 @@ import java.io.IOException;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.AsyncAdmin;
-import org.apache.hadoop.hbase.client.AsyncConnection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
@@ -81,7 +77,8 @@ public class TestClientClusterStatus {
   @Test
   public void testDefaults() throws Exception {
     ClusterStatus origin = ADMIN.getClusterStatus();
-    ClusterStatus defaults = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
+    ClusterStatus defaults
+        = new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.allOf(Option.class)));
     checkPbObjectNotNull(origin);
     checkPbObjectNotNull(defaults);
     Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
@@ -97,37 +94,16 @@ public class TestClientClusterStatus {
 
   @Test
   public void testNone() throws Exception {
-    ClusterStatus status0 = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
-    ClusterStatus status1 = ADMIN.getClusterStatus(EnumSet.noneOf(Option.class));
+    ClusterStatus status0
+      = new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.allOf(Option.class)));
+    ClusterStatus status1
+      = new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.noneOf(Option.class)));
     Assert.assertEquals(status0, status1);
     checkPbObjectNotNull(status0);
     checkPbObjectNotNull(status1);
   }
 
   @Test
-  public void testAsyncClient() throws Exception {
-    try (AsyncConnection asyncConnect = ConnectionFactory.createAsyncConnection(
-      UTIL.getConfiguration()).get()) {
-      AsyncAdmin asyncAdmin = asyncConnect.getAdmin();
-      CompletableFuture<ClusterStatus> originFuture =
-        asyncAdmin.getClusterStatus();
-      CompletableFuture<ClusterStatus> defaultsFuture =
-        asyncAdmin.getClusterStatus(EnumSet.allOf(Option.class));
-      ClusterStatus origin = originFuture.get();
-      ClusterStatus defaults = defaultsFuture.get();
-      checkPbObjectNotNull(origin);
-      checkPbObjectNotNull(defaults);
-      Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
-      Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
-      Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
-      Assert.assertTrue(origin.getBackupMastersSize() == defaults.getBackupMastersSize());
-      Assert.assertTrue(origin.getDeadServersSize() == defaults.getDeadServersSize());
-      Assert.assertTrue(origin.getRegionsCount() == defaults.getRegionsCount());
-      Assert.assertTrue(origin.getServersSize() == defaults.getServersSize());
-    }
-  }
-
-  @Test
   public void testLiveAndDeadServersStatus() throws Exception {
     // Count the number of live regionservers
     List<RegionServerThread> regionserverThreads = CLUSTER.getLiveRegionServerThreads();
@@ -143,14 +119,15 @@ public class TestClientClusterStatus {
     Waiter.waitFor(CLUSTER.getConfiguration(), 10 * 1000, 100, new Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        ClusterStatus status = ADMIN.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+        ClusterStatus status
+          = new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)));
         Assert.assertNotNull(status);
         return status.getRegionsCount() > 0;
       }
     });
     // Retrieve live servers and dead servers info.
     EnumSet<Option> options = EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS);
-    ClusterStatus status = ADMIN.getClusterStatus(options);
+    ClusterStatus status = new ClusterStatus(ADMIN.getClusterMetrics(options));
     checkPbObjectNotNull(status);
     Assert.assertNotNull(status);
     Assert.assertNotNull(status.getServers());
@@ -188,7 +165,7 @@ public class TestClientClusterStatus {
     Assert.assertEquals(MASTERS, masterThreads.size());
     // Retrieve master and backup masters infos only.
     EnumSet<Option> options = EnumSet.of(Option.MASTER, Option.BACKUP_MASTERS);
-    ClusterStatus status = ADMIN.getClusterStatus(options);
+    ClusterStatus status = new ClusterStatus(ADMIN.getClusterMetrics(options));
     Assert.assertTrue(status.getMaster().equals(activeName));
     Assert.assertEquals(MASTERS - 1, status.getBackupMastersSize());
   }
@@ -198,7 +175,7 @@ public class TestClientClusterStatus {
     EnumSet<Option> options =
         EnumSet.of(Option.MASTER_COPROCESSORS, Option.HBASE_VERSION,
                    Option.CLUSTER_ID, Option.BALANCER_ON);
-    ClusterStatus status = ADMIN.getClusterStatus(options);
+    ClusterStatus status = new ClusterStatus(ADMIN.getClusterMetrics(options));
     Assert.assertTrue(status.getMasterCoprocessors().length == 1);
     Assert.assertNotNull(status.getHBaseVersion());
     Assert.assertNotNull(status.getClusterId());
@@ -245,13 +222,13 @@ public class TestClientClusterStatus {
       return Optional.of(this);
     }
 
-    @Override public void preGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx)
+    @Override public void preGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx)
         throws IOException {
       PRE_COUNT.incrementAndGet();
     }
 
-    @Override public void postGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        ClusterStatus status) throws IOException {
+    @Override public void postGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      ClusterMetrics status) throws IOException {
       POST_COUNT.incrementAndGet();
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
index e62d212..c1dab38 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
@@ -28,6 +28,8 @@ import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Table;
@@ -83,9 +85,10 @@ public class TestRegionLoad {
 
     // Check if regions match with the regionLoad from the server
     for (ServerName serverName : admin
-        .getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
+        .getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()) {
       List<HRegionInfo> regions = admin.getOnlineRegions(serverName);
-      Collection<RegionLoad> regionLoads = admin.getRegionLoad(serverName).values();
+      Collection<RegionLoad> regionLoads = admin.getRegionMetrics(serverName)
+        .stream().map(r -> new RegionLoad(r)).collect(Collectors.toList());
       checkRegionsAndRegionLoads(regions, regionLoads);
     }
 
@@ -95,17 +98,23 @@ public class TestRegionLoad {
 
       List<RegionLoad> regionLoads = Lists.newArrayList();
       for (ServerName serverName : admin
-          .getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
-        regionLoads.addAll(admin.getRegionLoad(serverName, table).values());
+          .getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()) {
+        regionLoads.addAll(admin.getRegionMetrics(serverName, table)
+          .stream().map(r -> new RegionLoad(r)).collect(Collectors.toList()));
       }
       checkRegionsAndRegionLoads(tableRegions, regionLoads);
     }
 
     // Check RegionLoad matches the regionLoad from ClusterStatus
-    ClusterStatus clusterStatus = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+    ClusterStatus clusterStatus
+      = new ClusterStatus(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)));
     for (ServerName serverName : clusterStatus.getServers()) {
       ServerLoad serverLoad = clusterStatus.getLoad(serverName);
-      Map<byte[], RegionLoad> regionLoads = admin.getRegionLoad(serverName);
+      Map<byte[], RegionLoad> regionLoads = admin.getRegionMetrics(serverName).stream()
+        .collect(Collectors.toMap(e -> e.getRegionName(), e -> new RegionLoad(e),
+          (v1, v2) -> {
+            throw new RuntimeException("impossible!!");
+          }, () -> new TreeMap<>(Bytes.BYTES_COMPARATOR)));
       compareRegionLoads(serverLoad.getRegionsLoad(), regionLoads);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java
index 329b544..34196e2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java
@@ -27,7 +27,6 @@ import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -79,11 +78,11 @@ public class TestRegionMetrics {
   public void testRegionMetrics() throws Exception {
 
     // Check if regions match with the RegionMetrics from the server
-    for (ServerName serverName : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
+    for (ServerName serverName : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
         .getLiveServerMetrics().keySet()) {
       List<RegionInfo> regions = admin.getRegions(serverName);
       Collection<RegionMetrics> regionMetricsList =
-          admin.getRegionLoads(serverName).stream().collect(Collectors.toList());
+          admin.getRegionMetrics(serverName);
       checkRegionsAndRegionMetrics(regions, regionMetricsList);
     }
 
@@ -92,21 +91,20 @@ public class TestRegionMetrics {
       List<RegionInfo> tableRegions = admin.getRegions(table);
 
       List<RegionMetrics> regionMetrics = new ArrayList<>();
-      for (ServerName serverName : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
+      for (ServerName serverName : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
           .getLiveServerMetrics().keySet()) {
-        regionMetrics.addAll(admin.getRegionLoads(serverName, table));
+        regionMetrics.addAll(admin.getRegionMetrics(serverName, table));
       }
       checkRegionsAndRegionMetrics(tableRegions, regionMetrics);
     }
 
     // Check RegionMetrics matches the RegionMetrics from ClusterStatus
-    ClusterMetrics clusterStatus = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+    ClusterMetrics clusterStatus = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS));
     for (Map.Entry<ServerName, ServerMetrics> entry : clusterStatus.getLiveServerMetrics()
         .entrySet()) {
       ServerName serverName = entry.getKey();
       ServerMetrics serverMetrics = entry.getValue();
-      List<RegionMetrics> regionMetrics =
-          admin.getRegionLoads(serverName).stream().collect(Collectors.toList());
+      List<RegionMetrics> regionMetrics = admin.getRegionMetrics(serverName);
       assertEquals(serverMetrics.getRegionMetrics().size(), regionMetrics.size());
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/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
index 22ab8b9..c5e6464 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
@@ -20,13 +20,6 @@ package org.apache.hadoop.hbase;
 
 import static org.junit.Assert.assertTrue;
 
-import javax.management.MBeanAttributeInfo;
-import javax.management.MBeanInfo;
-import javax.management.MBeanServerConnection;
-import javax.management.ObjectInstance;
-import javax.management.ObjectName;
-import javax.management.remote.JMXConnector;
-import javax.management.remote.JMXConnectorFactory;
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.Hashtable;
@@ -35,7 +28,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
-
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanInfo;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectInstance;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@@ -156,7 +155,7 @@ public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
     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
+    // NOTE the size is normally set in setClusterMetrics, for test purpose, we set it manually
     // Tables: hbase:namespace, table1, table2
     // Functions: costFunctions, overall
     String[] functionNames = loadBalancer.getCostFunctionNames();

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index d701a81..05b8edc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -698,7 +698,8 @@ public class TestAdmin2 {
     TEST_UTIL.createMultiRegionTable(tableName, "f".getBytes(), 6);
 
     ArrayList<ServerName> clusterRegionServers =
-        new ArrayList<>(admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers());
+        new ArrayList<>(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+          .getLiveServerMetrics().keySet());
 
     assertEquals(clusterRegionServers.size(), 3);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
index e0040b8..5b33ad0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
@@ -32,12 +32,12 @@ import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.RegionLoad;
-import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerMetrics;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -244,38 +244,43 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
     Collection<ServerName> servers = admin.getRegionServers().get();
     for (ServerName serverName : servers) {
       List<RegionInfo> regions = admin.getRegions(serverName).get();
-      checkRegionsAndRegionLoads(regions, admin.getRegionLoads(serverName).get());
+      checkRegionsAndRegionLoads(regions, admin.getRegionMetrics(serverName).get());
     }
 
     // Check if regionLoad matches the table's regions and nothing is missed
     for (TableName table : tables) {
       List<RegionInfo> tableRegions = admin.getRegions(table).get();
-      List<RegionLoad> regionLoads = Lists.newArrayList();
+      List<RegionMetrics> regionLoads = Lists.newArrayList();
       for (ServerName serverName : servers) {
-        regionLoads.addAll(admin.getRegionLoads(serverName, table).get());
+        regionLoads.addAll(admin.getRegionMetrics(serverName, table).get());
       }
       checkRegionsAndRegionLoads(tableRegions, regionLoads);
     }
 
     // Check RegionLoad matches the regionLoad from ClusterStatus
-    ClusterStatus clusterStatus = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).get();
-    for (ServerName serverName : clusterStatus.getServers()) {
-      ServerLoad serverLoad = clusterStatus.getLoad(serverName);
-      compareRegionLoads(serverLoad.getRegionsLoad().values(), admin.getRegionLoads(serverName)
-          .get());
+    ClusterMetrics clusterStatus = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).get();
+    for (Map.Entry<ServerName, ServerMetrics> entry :
+      clusterStatus.getLiveServerMetrics().entrySet()) {
+      ServerName sn = entry.getKey();
+      ServerMetrics sm = entry.getValue();
+      compareRegionLoads(sm.getRegionMetrics().values(), admin.getRegionMetrics(sn).get());
+    }
+    for (ServerName serverName : clusterStatus.getLiveServerMetrics().keySet()) {
+      ServerMetrics serverLoad = clusterStatus.getLiveServerMetrics().get(serverName);
+
     }
   }
 
-  private void compareRegionLoads(Collection<RegionLoad> regionLoadCluster,
-      Collection<RegionLoad> regionLoads) {
+  private void compareRegionLoads(Collection<RegionMetrics> regionLoadCluster,
+      Collection<RegionMetrics> regionLoads) {
 
     assertEquals("No of regionLoads from clusterStatus and regionloads from RS doesn't match",
       regionLoadCluster.size(), regionLoads.size());
 
-    for (RegionLoad loadCluster : regionLoadCluster) {
+    for (RegionMetrics loadCluster : regionLoadCluster) {
       boolean matched = false;
-      for (RegionLoad load : regionLoads) {
-        if (Bytes.equals(loadCluster.getName(), load.getName())) {
+      for (RegionMetrics load : regionLoads) {
+        if (Bytes.equals(loadCluster.getRegionName(), load.getRegionName())) {
           matched = true;
           continue;
         }
@@ -285,13 +290,13 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
   }
 
   private void checkRegionsAndRegionLoads(Collection<RegionInfo> regions,
-      Collection<RegionLoad> regionLoads) {
+      Collection<RegionMetrics> regionLoads) {
 
     assertEquals("No of regions and regionloads doesn't match", regions.size(), regionLoads.size());
 
-    Map<byte[], RegionLoad> regionLoadMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
-    for (RegionLoad regionLoad : regionLoads) {
-      regionLoadMap.put(regionLoad.getName(), regionLoad);
+    Map<byte[], RegionMetrics> regionLoadMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
+    for (RegionMetrics regionLoad : regionLoads) {
+      regionLoadMap.put(regionLoad.getRegionName(), regionLoad);
     }
     for (RegionInfo info : regions) {
       assertTrue("Region not in regionLoadMap region:" + info.getRegionNameAsString()

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java
index a3d7616..8c2b060 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java
@@ -46,7 +46,8 @@ public class TestAsyncDecommissionAdminApi extends TestAsyncAdminBase {
     TEST_UTIL.createMultiRegionTable(tableName, FAMILY, 4);
 
     ArrayList<ServerName> clusterRegionServers =
-        new ArrayList<>(admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).get().getServers());
+        new ArrayList<>(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).get()
+          .getLiveServerMetrics().keySet());
 
     assertEquals(clusterRegionServers.size(), 2);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index e48667e..f17c291 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -44,7 +44,6 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
@@ -58,10 +57,10 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -4261,8 +4260,8 @@ public class TestFromClientSide {
     boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
     try (Admin admin = conn.getAdmin()) {
       assertTrue(admin.tableExists(tableName));
-      assertTrue(admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
-          .getServersSize() == SLAVES + (tablesOnMaster ? 1 : 0));
+      assertTrue(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+          .getLiveServerMetrics().size() == SLAVES + (tablesOnMaster ? 1 : 0));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 4497a529..e01c28f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -23,13 +23,13 @@ import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import edu.umd.cs.findbugs.annotations.Nullable;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
@@ -68,8 +68,6 @@ import org.junit.rules.TestRule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import edu.umd.cs.findbugs.annotations.Nullable;
-
 /**
  * Tests the scenarios where replicas are enabled for the meta table
  */
@@ -207,7 +205,7 @@ public class TestMetaWithReplicas {
           Thread.sleep(conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD,
               30000) * 3);
         }
-        master = util.getHBaseClusterInterface().getClusterStatus().getMaster();
+        master = util.getHBaseClusterInterface().getClusterMetrics().getMasterName();
         // kill the master so that regionserver recovery is not triggered at all
         // for the meta server
         util.getHBaseClusterInterface().stopMaster(master);
@@ -276,7 +274,7 @@ public class TestMetaWithReplicas {
   private void stopMasterAndValidateReplicaCount(final int originalReplicaCount,
       final int newReplicaCount)
       throws Exception {
-    ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterStatus().getMaster();
+    ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterMetrics().getMasterName();
     TEST_UTIL.getHBaseClusterInterface().stopMaster(sn);
     TEST_UTIL.getHBaseClusterInterface().waitForMasterToStop(sn, 60000);
     List<String> metaZnodes = TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes();
@@ -409,7 +407,7 @@ public class TestMetaWithReplicas {
     byte[] data = ZKUtil.getData(zkw, primaryMetaZnode);
     ServerName currentServer = ProtobufUtil.toServerName(data);
     Collection<ServerName> liveServers = TEST_UTIL.getAdmin()
-        .getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers();
+        .getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet();
     ServerName moveToServer = null;
     for (ServerName s : liveServers) {
       if (!currentServer.equals(s)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
index 8e44640..49e0a98 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
@@ -336,7 +336,7 @@ public class TestMultiParallel {
         public boolean evaluate() throws Exception {
           // Master is also a regionserver, so the count is liveRScount
           return UTIL.getMiniHBaseCluster().getMaster()
-              .getClusterStatus().getServersSize() == liveRScount;
+              .getClusterMetrics().getLiveServerMetrics().size() == liveRScount;
         }
       });
       UTIL.waitFor(15 * 1000, UTIL.predicateNoRegionsInTransition());

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 0309eaa..431e73e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -29,7 +29,6 @@ import java.util.List;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -70,6 +69,7 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
@@ -1523,7 +1523,7 @@ public class TestMasterObserver {
       assertNotNull("Found a non-null entry", firstGoodPair);
       LOG.info("Found " + firstGoodPair.toString());
       // Try to force a move
-      Collection<ServerName> servers = master.getClusterStatus().getServers();
+      Collection<ServerName> servers = master.getClusterMetrics().getLiveServerMetrics().keySet();
       String destName = null;
       String serverNameForFirstRegion = firstGoodPair.getServerName().toString();
       LOG.info("serverNameForFirstRegion=" + serverNameForFirstRegion);

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
index ff813eb..e8aa755 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
@@ -23,9 +23,8 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.util.List;
-
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
@@ -97,10 +96,9 @@ public class TestMasterFailover {
 
       // Check that ClusterStatus reports the correct active and backup masters
       assertNotNull(active);
-      ClusterStatus status = active.getClusterStatus();
-      assertTrue(status.getMaster().equals(activeName));
-      assertEquals(2, status.getBackupMastersSize());
-      assertEquals(2, status.getBackupMasters().size());
+      ClusterMetrics status = active.getClusterMetrics();
+      assertTrue(status.getMasterName().equals(activeName));
+      assertEquals(2, status.getBackupMasterNames().size());
 
       // attempt to stop one of the inactive masters
       int backupIndex = (activeIndex == 0 ? 1 : activeIndex - 1);
@@ -119,17 +117,17 @@ public class TestMasterFailover {
       }
       assertEquals(1, numActive);
       assertEquals(2, masterThreads.size());
-      int rsCount = masterThreads.get(activeIndex).getMaster().getClusterStatus().getServersSize();
+      int rsCount = masterThreads.get(activeIndex).getMaster().getClusterMetrics()
+        .getLiveServerMetrics().size();
       LOG.info("Active master " + active.getServerName() + " managing " + rsCount +
           " regions servers");
       assertEquals(3, rsCount);
 
       // Check that ClusterStatus reports the correct active and backup masters
       assertNotNull(active);
-      status = active.getClusterStatus();
-      assertTrue(status.getMaster().equals(activeName));
-      assertEquals(1, status.getBackupMastersSize());
-      assertEquals(1, status.getBackupMasters().size());
+      status = active.getClusterMetrics();
+      assertTrue(status.getMasterName().equals(activeName));
+      assertEquals(1, status.getBackupMasterNames().size());
 
       // kill the active master
       LOG.debug("\n\nStopping the active master " + active.getServerName() + "\n");
@@ -146,13 +144,12 @@ public class TestMasterFailover {
       // and he should be active
       active = masterThreads.get(0).getMaster();
       assertNotNull(active);
-      status = active.getClusterStatus();
-      ServerName mastername = status.getMaster();
+      status = active.getClusterMetrics();
+      ServerName mastername = status.getMasterName();
       assertTrue(mastername.equals(active.getServerName()));
       assertTrue(active.isActiveMaster());
-      assertEquals(0, status.getBackupMastersSize());
-      assertEquals(0, status.getBackupMasters().size());
-      int rss = status.getServersSize();
+      assertEquals(0, status.getBackupMasterNames().size());
+      int rss = status.getLiveServerMetrics().size();
       LOG.info("Active master " + mastername.getServerName() + " managing " +
           rss + " region servers");
       assertEquals(3, rss);

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailoverBalancerPersistence.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailoverBalancerPersistence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailoverBalancerPersistence.java
index 395fc31..4a451ba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailoverBalancerPersistence.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailoverBalancerPersistence.java
@@ -18,7 +18,12 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import org.apache.hadoop.hbase.ClusterStatus;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -28,12 +33,6 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.IOException;
-import java.util.List;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 @Category({MasterTests.class, LargeTests.class})
 public class TestMasterFailoverBalancerPersistence {
 
@@ -57,14 +56,14 @@ public class TestMasterFailoverBalancerPersistence {
     assertTrue(cluster.waitForActiveAndReadyMaster());
     HMaster active = cluster.getMaster();
     // check that the balancer is on by default for the active master
-    ClusterStatus clusterStatus = active.getClusterStatus();
-    assertTrue(clusterStatus.isBalancerOn());
+    ClusterMetrics clusterStatus = active.getClusterMetrics();
+    assertTrue(clusterStatus.getBalancerOn());
 
     active = killActiveAndWaitForNewActive(cluster);
 
     // ensure the load balancer is still running on new master
-    clusterStatus = active.getClusterStatus();
-    assertTrue(clusterStatus.isBalancerOn());
+    clusterStatus = active.getClusterMetrics();
+    assertTrue(clusterStatus.getBalancerOn());
 
     // turn off the load balancer
     active.balanceSwitch(false);
@@ -73,8 +72,8 @@ public class TestMasterFailoverBalancerPersistence {
     active = killActiveAndWaitForNewActive(cluster);
 
     // ensure the load balancer is not running on the new master
-    clusterStatus = active.getClusterStatus();
-    assertFalse(clusterStatus.isBalancerOn());
+    clusterStatus = active.getClusterMetrics();
+    assertFalse(clusterStatus.getBalancerOn());
 
     // Stop the cluster
     TEST_UTIL.shutdownMiniCluster();

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
index 13d9841..cd7af50 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
@@ -84,8 +84,8 @@ public class TestMasterOperationsForRegionReplicas {
     TEST_UTIL.startMiniCluster(numSlaves);
     CONNECTION = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
     ADMIN = CONNECTION.getAdmin();
-    while(ADMIN.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
-               .getServers().size() < numSlaves) {
+    while(ADMIN.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+               .getLiveServerMetrics().size() < numSlaves) {
       Thread.sleep(100);
     }
   }
@@ -163,7 +163,7 @@ public class TestMasterOperationsForRegionReplicas {
         ADMIN.getConnection());
 
       // Now kill the master, restart it and see if the assignments are kept
-      ServerName master = TEST_UTIL.getHBaseClusterInterface().getClusterStatus().getMaster();
+      ServerName master = TEST_UTIL.getHBaseClusterInterface().getClusterMetrics().getMasterName();
       TEST_UTIL.getHBaseClusterInterface().stopMaster(master);
       TEST_UTIL.getHBaseClusterInterface().waitForMasterToStop(master, 30000);
       TEST_UTIL.getHBaseClusterInterface().startMaster(master.getHostname(), master.getPort());

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
index 0eff8df..29c24f7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
@@ -24,9 +24,8 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.LocalHBaseCluster;
@@ -84,9 +83,8 @@ public class TestMasterShutdown {
     }
     assertNotNull(active);
     // make sure the other two are backup masters
-    ClusterStatus status = active.getClusterStatus();
-    assertEquals(2, status.getBackupMastersSize());
-    assertEquals(2, status.getBackupMasters().size());
+    ClusterMetrics status = active.getClusterMetrics();
+    assertEquals(2, status.getBackupMasterNames().size());
 
     // tell the active master to shutdown the cluster
     active.shutdown();

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
index a053f7d..67ed330 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
@@ -28,8 +28,8 @@ import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -169,14 +169,14 @@ public class TestFavoredStochasticBalancerPickers extends BalancerTestBase {
     TEST_UTIL.getHBaseCluster().startRegionServerAndWait(60000);
 
     Map<ServerName, List<RegionInfo>> serverAssignments = Maps.newHashMap();
-    ClusterStatus status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
-    for (ServerName sn : status.getServers()) {
+    ClusterMetrics status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS));
+    for (ServerName sn : status.getLiveServerMetrics().keySet()) {
       if (!ServerName.isSameAddress(sn, masterServerName)) {
         serverAssignments.put(sn, getTableRegionsFromServer(tableName, sn));
       }
     }
     RegionLocationFinder regionFinder = new RegionLocationFinder();
-    regionFinder.setClusterStatus(admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)));
+    regionFinder.setClusterMetrics(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)));
     regionFinder.setConf(conf);
     regionFinder.setServices(TEST_UTIL.getMiniHBaseCluster().getMaster());
     Cluster cluster = new Cluster(serverAssignments, null, regionFinder, new RackManager(conf));

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
index 8862ea0..89fe35a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
@@ -30,6 +30,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -154,7 +155,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     regions.addAll(admin.getTableRegions(TableName.META_TABLE_NAME));
     regions.addAll(admin.getTableRegions(TableName.NAMESPACE_TABLE_NAME));
     List<ServerName> servers = Lists.newArrayList(
-      admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers());
+      admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet());
     Map<ServerName, List<RegionInfo>> map = balancer.roundRobinAssignment(regions, servers);
     for (List<RegionInfo> regionInfos : map.values()) {
       regions.removeAll(regionInfos);
@@ -181,11 +182,11 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     }
 
     Map<ServerName, List<Integer>> replicaLoadMap = fnm.getReplicaLoad(
-      Lists.newArrayList(admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
-                              .getServers()));
+      Lists.newArrayList(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+                              .getLiveServerMetrics().keySet()));
     assertTrue("Not all replica load collected.",
-      admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
-           .getServers().size() == replicaLoadMap.size());
+      admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+           .getLiveServerMetrics().size() == replicaLoadMap.size());
     for (Entry<ServerName, List<Integer>> entry : replicaLoadMap.entrySet()) {
       assertTrue(entry.getValue().size() == FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
       assertTrue(entry.getValue().get(0) >= 0);
@@ -196,10 +197,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.disableTable(TableName.valueOf(tableName));
     admin.deleteTable(TableName.valueOf(tableName));
     replicaLoadMap = fnm.getReplicaLoad(Lists.newArrayList(
-      admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()));
+      admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()));
     assertTrue("replica load found " + replicaLoadMap.size() + " instead of 0.",
       replicaLoadMap.size() == admin
-          .getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()
+          .getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics()
           .size());
   }
 
@@ -220,7 +221,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
     LoadBalancer balancer = master.getLoadBalancer();
     ServerName destination = balancer.randomAssignment(hri, Lists.newArrayList(admin
-        .getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()));
+        .getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics()
+        .keySet().stream().collect(Collectors.toList())));
     assertNotNull(destination);
     List<ServerName> favoredNodes = fnm.getFavoredNodes(hri);
     assertNotNull(favoredNodes);
@@ -286,7 +288,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     assertNotNull(currentFN);
 
     List<ServerName> serversForNewFN = Lists.newArrayList();
-    for (ServerName sn : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
+    for (ServerName sn : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+      .getLiveServerMetrics().keySet()) {
       serversForNewFN.add(ServerName.valueOf(sn.getHostname(), sn.getPort(), NON_STARTCODE));
     }
     for (ServerName sn : currentFN) {
@@ -386,7 +389,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
     // Regenerate FN and assign, everything else should be fine
     List<ServerName> serversForNewFN = Lists.newArrayList();
-    for (ServerName sn : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
+    for (ServerName sn : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+      .getLiveServerMetrics().keySet()) {
       serversForNewFN.add(ServerName.valueOf(sn.getHostname(), sn.getPort(), NON_STARTCODE));
     }
 
@@ -480,7 +484,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
     // Regenerate FN and assign, everything else should be fine
     List<ServerName> serversForNewFN = Lists.newArrayList();
-    for (ServerName sn : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
+    for (ServerName sn : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+      .getLiveServerMetrics().keySet()) {
       serversForNewFN.add(ServerName.valueOf(sn.getHostname(), sn.getPort(), NON_STARTCODE));
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
index 0b49b5e..0479065 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -69,7 +68,7 @@ public class TestRegionLocationFinder {
 
     finder.setConf(TEST_UTIL.getConfiguration());
     finder.setServices(cluster.getMaster());
-    finder.setClusterStatus(cluster.getMaster().getClusterStatus());
+    finder.setClusterMetrics(cluster.getMaster().getClusterMetrics());
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
index 1e6f1dd..a326d9a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
@@ -25,7 +25,6 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -33,13 +32,13 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Queue;
 import java.util.TreeMap;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.RegionLoad;
-import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerMetrics;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Size;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.master.MockNoopMasterServices;
@@ -126,20 +125,25 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
     ServerName sn = ServerName.valueOf("test:8080", 100);
     int numClusterStatusToAdd = 20000;
     for (int i = 0; i < numClusterStatusToAdd; i++) {
-      ServerLoad sl = mock(ServerLoad.class);
+      ServerMetrics sl = mock(ServerMetrics.class);
 
-      RegionLoad rl = mock(RegionLoad.class);
-      when(rl.getStorefileSizeMB()).thenReturn(i);
+      RegionMetrics rl = mock(RegionMetrics.class);
+      when(rl.getReadRequestCount()).thenReturn(0L);
+      when(rl.getWriteRequestCount()).thenReturn(0L);
+      when(rl.getMemStoreSize()).thenReturn(Size.ZERO);
+      when(rl.getStoreFileSize()).thenReturn(new Size(i, Size.Unit.MEGABYTE));
 
-      Map<byte[], RegionLoad> regionLoadMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+      Map<byte[], RegionMetrics> regionLoadMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
       regionLoadMap.put(Bytes.toBytes(REGION_KEY), rl);
-      when(sl.getRegionsLoad()).thenReturn(regionLoadMap);
+      when(sl.getRegionMetrics()).thenReturn(regionLoadMap);
 
-      ClusterStatus clusterStatus = mock(ClusterStatus.class);
-      when(clusterStatus.getServers()).thenReturn(Arrays.asList(sn));
-      when(clusterStatus.getLoad(sn)).thenReturn(sl);
+      ClusterMetrics clusterStatus = mock(ClusterMetrics.class);
+      Map<ServerName, ServerMetrics> serverMetricsMap = new TreeMap<>();
+      serverMetricsMap.put(sn, sl);
+      when(clusterStatus.getLiveServerMetrics()).thenReturn(serverMetricsMap);
+//      when(clusterStatus.getLoad(sn)).thenReturn(sl);
 
-      loadBalancer.setClusterStatus(clusterStatus);
+      loadBalancer.setClusterMetrics(clusterStatus);
     }
     assertTrue(loadBalancer.loads.get(REGION_KEY) != null);
     assertTrue(loadBalancer.loads.get(REGION_KEY).size() == 15);

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
index e7b35d9..536fdb2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
@@ -106,7 +106,8 @@ public class TestRegionServerReadRequestMetrics {
     TEST_UTIL.getConfiguration().setBoolean(LoadBalancer.SYSTEM_TABLES_ON_MASTER, true);
     TEST_UTIL.startMiniCluster();
     admin = TEST_UTIL.getAdmin();
-    serverNames = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers();
+    serverNames = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+      .getLiveServerMetrics().keySet();
     table = createTable();
     putData();
     List<RegionInfo> regions = admin.getRegions(TABLE_NAME);
@@ -166,7 +167,8 @@ public class TestRegionServerReadRequestMetrics {
     boolean metricsUpdated = false;
     for (int i = 0; i < MAX_TRY; i++) {
       for (ServerName serverName : serverNames) {
-        serverLoad = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getLoad(serverName);
+        serverLoad = new ServerLoad(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+          .getLiveServerMetrics().get(serverName));
 
         Map<byte[], RegionLoad> regionsLoad = serverLoad.getRegionsLoad();
         RegionLoad regionLoad = regionsLoad.get(regionInfo.getRegionName());
@@ -442,8 +444,8 @@ public class TestRegionServerReadRequestMetrics {
 
   private void testReadRequests(byte[] regionName, int expectedReadRequests) throws Exception {
     for (ServerName serverName : serverNames) {
-      ServerLoad serverLoad =
-          admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getLoad(serverName);
+      ServerLoad serverLoad = new ServerLoad(admin.getClusterMetrics(
+        EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().get(serverName));
       Map<byte[], RegionLoad> regionsLoad = serverLoad.getRegionsLoad();
       RegionLoad regionLoad = regionsLoad.get(regionName);
       if (regionLoad != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 4c66b79..ab2d4b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -33,7 +33,6 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -904,13 +903,14 @@ public class TestSplitTransactionOnCluster {
 
   private void waitUntilRegionServerDead() throws InterruptedException, IOException {
     // Wait until the master processes the RS shutdown
-    for (int i=0; (cluster.getMaster().getClusterStatus().getServers().size() > NB_SERVERS
+    for (int i=0; (cluster.getMaster().getClusterMetrics()
+        .getLiveServerMetrics().size() > NB_SERVERS
         || cluster.getLiveRegionServerThreads().size() > NB_SERVERS) && i<100; i++) {
       LOG.info("Waiting on server to go down");
       Thread.sleep(100);
     }
     assertFalse("Waited too long for RS to die",
-      cluster.getMaster().getClusterStatus(). getServers().size() > NB_SERVERS
+      cluster.getMaster().getClusterMetrics(). getLiveServerMetrics().size() > NB_SERVERS
         || cluster.getLiveRegionServerThreads().size() > NB_SERVERS);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
index df929c8..1012720 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
@@ -35,13 +35,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.ClusterMetrics;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerMetrics;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
@@ -187,9 +186,9 @@ public class TestMasterReplication {
     Waiter.waitFor(baseConfiguration, 10000, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        ClusterStatus clusterStatus = utilities[0].getAdmin()
-            .getClusterStatus(EnumSet.of(ClusterMetrics.Option.LIVE_SERVERS));
-        ServerLoad serverLoad = clusterStatus.getLoad(rsName);
+        ClusterMetrics clusterStatus = utilities[0].getAdmin()
+            .getClusterMetrics(EnumSet.of(ClusterMetrics.Option.LIVE_SERVERS));
+        ServerMetrics serverLoad = clusterStatus.getLiveServerMetrics().get(rsName);
         List<ReplicationLoadSource> replicationLoadSourceList =
             serverLoad.getReplicationLoadSourceList();
         return replicationLoadSourceList.isEmpty();

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
index f76be44..7a624b2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
@@ -67,7 +67,8 @@ public class TestReplicationStatus extends TestReplicationBase {
         htable1.put(p);
       }
 
-      ClusterStatus status = hbaseAdmin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+      ClusterStatus status = new ClusterStatus(hbaseAdmin.getClusterMetrics(
+        EnumSet.of(Option.LIVE_SERVERS)));
 
       for (JVMClusterUtil.RegionServerThread thread : utility1.getHBaseCluster()
           .getRegionServerThreads()) {
@@ -90,7 +91,7 @@ public class TestReplicationStatus extends TestReplicationBase {
       // Stop rs1, then the queue of rs1 will be transfered to rs0
       utility1.getHBaseCluster().getRegionServer(1).stop("Stop RegionServer");
       Thread.sleep(10000);
-      status = hbaseAdmin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+      status = new ClusterStatus(hbaseAdmin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)));
       ServerName server = utility1.getHBaseCluster().getRegionServer(0).getServerName();
       ServerLoad sl = status.getLoad(server);
       List<ReplicationLoadSource> rLoadSourceList = sl.getReplicationLoadSourceList();

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 83a6dfd..2435532 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -120,8 +120,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
-import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
@@ -137,6 +135,9 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
+
 /**
  * Performs authorization checks for common operations, according to different
  * levels of authorized users.
@@ -3129,7 +3130,7 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        ACCESS_CONTROLLER.preGetClusterStatus(ObserverContextImpl.createAndPrepare(CP_ENV));
+        ACCESS_CONTROLLER.preGetClusterMetrics(ObserverContextImpl.createAndPrepare(CP_ENV));
         return null;
       }
     };

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
index bb52b5f..5aca352 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import com.google.protobuf.ServiceException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -29,7 +30,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -72,14 +71,13 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
 
-import com.google.protobuf.ServiceException;
-
 /**
  * Utilities class for snapshots
  */
@@ -229,9 +227,6 @@ public final class SnapshotTestingUtils {
         }
       });
     }
-    for (byte[] b : snapshotFamilies) {
-      LOG.info("[CHIA] " + Bytes.toStringBinary(b));
-    }
     // Verify that there are store files in the specified families
     if (nonEmptyTestFamilies != null) {
       for (final byte[] familyName: nonEmptyTestFamilies) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index 9e4de7e..87d718e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
@@ -37,14 +37,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
@@ -54,7 +53,6 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
@@ -319,8 +317,8 @@ public class BaseTestHBaseFsck {
    * Get region info from local cluster.
    */
   Map<ServerName, List<String>> getDeployedHRIs(final Admin admin) throws IOException {
-    ClusterStatus status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
-    Collection<ServerName> regionServers = status.getServers();
+    ClusterMetrics status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS));
+    Collection<ServerName> regionServers = status.getLiveServerMetrics().keySet();
     Map<ServerName, List<String>> mm = new HashMap<>();
     for (ServerName hsi : regionServers) {
       AdminProtos.AdminService.BlockingInterface server = connection.getAdmin(hsi);

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
index 0b9c4b5..3230e27 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
@@ -153,8 +153,8 @@ public class TestMiniClusterLoadSequential {
         ", isMultiPut=" + isMultiPut);
     numKeys = numKeys();
     Admin admin = TEST_UTIL.getAdmin();
-    while (admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
-                .getServers().size() < NUM_RS) {
+    while (admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+                .getLiveServerMetrics().size() < NUM_RS) {
       LOG.info("Sleeping until " + NUM_RS + " RSs are online");
       Threads.sleepWithoutInterrupt(1000);
     }


[2/3] hbase git commit: HBASE-19596 RegionMetrics/ServerMetrics/ClusterMetrics should apply to all public classes

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java
index 301cfef..1227595 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java
@@ -17,31 +17,28 @@
  */
 package org.apache.hadoop.hbase.mapreduce;
 
+import static org.apache.hadoop.hbase.HConstants.DEFAULT_REGIONSERVER_PORT;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.RegionLoad;
+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.Admin;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import static org.apache.hadoop.hbase.HConstants.DEFAULT_REGIONSERVER_PORT;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.when;
-
 @Category({MiscTests.class, SmallTests.class})
 public class TestRegionSizeCalculator {
 
@@ -134,14 +131,15 @@ public class TestRegionSizeCalculator {
   /**
    * Creates mock returning RegionLoad info about given servers.
   */
-  private Admin mockAdmin(RegionLoad... regionLoadArray) throws Exception {
+  private Admin mockAdmin(RegionMetrics... regionLoadArray) throws Exception {
     Admin mockAdmin = Mockito.mock(Admin.class);
-    Map<byte[], RegionLoad> regionLoads = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for (RegionLoad regionLoad : regionLoadArray) {
-      regionLoads.put(regionLoad.getName(), regionLoad);
+    List<RegionMetrics> regionLoads = new ArrayList<>();
+    for (RegionMetrics regionLoad : regionLoadArray) {
+      regionLoads.add(regionLoad);
     }
     when(mockAdmin.getConfiguration()).thenReturn(configuration);
-    when(mockAdmin.getRegionLoad(sn, TableName.valueOf("sizeTestTable"))).thenReturn(regionLoads);
+    when(mockAdmin.getRegionMetrics(sn, TableName.valueOf("sizeTestTable")))
+        .thenReturn(regionLoads);
     return mockAdmin;
   }
 
@@ -150,11 +148,11 @@ public class TestRegionSizeCalculator {
    *
    * @param  fileSizeMb number of megabytes occupied by region in file store in megabytes
    * */
-  private RegionLoad mockRegion(String regionName, int fileSizeMb) {
-    RegionLoad region = Mockito.mock(RegionLoad.class);
-    when(region.getName()).thenReturn(regionName.getBytes());
+  private RegionMetrics mockRegion(String regionName, int fileSizeMb) {
+    RegionMetrics region = Mockito.mock(RegionMetrics.class);
+    when(region.getRegionName()).thenReturn(regionName.getBytes());
     when(region.getNameAsString()).thenReturn(regionName);
-    when(region.getStorefileSizeMB()).thenReturn(fileSizeMb);
+    when(region.getStoreFileSize()).thenReturn(new Size(fileSizeMb, Size.Unit.MEGABYTE));
     return region;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java
index 7ee1065..2323bf3 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.rest;
 
 import java.io.IOException;
 import java.util.EnumSet;
+import java.util.Map;
 import javax.ws.rs.GET;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.CacheControl;
@@ -28,11 +29,12 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.UriInfo;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.RegionLoad;
-import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerMetrics;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Size;
 import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -67,29 +69,35 @@ public class StorageClusterStatusResource extends ResourceBase {
     }
     servlet.getMetrics().incrementRequests(1);
     try {
-      ClusterStatus status = servlet.getAdmin().getClusterStatus(
+      ClusterMetrics status = servlet.getAdmin().getClusterMetrics(
         EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS));
       StorageClusterStatusModel model = new StorageClusterStatusModel();
-      model.setRegions(status.getRegionsCount());
+      model.setRegions(status.getRegionCount());
       model.setRequests(status.getRequestCount());
       model.setAverageLoad(status.getAverageLoad());
-      for (ServerName info: status.getServers()) {
-        ServerLoad load = status.getLoad(info);
+      for (Map.Entry<ServerName, ServerMetrics> entry: status.getLiveServerMetrics().entrySet()) {
+        ServerName sn = entry.getKey();
+        ServerMetrics load = entry.getValue();
         StorageClusterStatusModel.Node node =
           model.addLiveNode(
-            info.getHostname() + ":" +
-            Integer.toString(info.getPort()),
-            info.getStartcode(), load.getUsedHeapMB(),
-            load.getMaxHeapMB());
-        node.setRequests(load.getNumberOfRequests());
-        for (RegionLoad region: load.getRegionsLoad().values()) {
-          node.addRegion(region.getName(), region.getStores(),
-            region.getStorefiles(), region.getStorefileSizeMB(),
-            region.getMemStoreSizeMB(), region.getStorefileIndexSizeKB(),
-            region.getReadRequestsCount(), region.getWriteRequestsCount(),
-            region.getRootIndexSizeKB(), region.getTotalStaticIndexSizeKB(),
-            region.getTotalStaticBloomSizeKB(), region.getTotalCompactingKVs(),
-            region.getCurrentCompactedKVs());
+            sn.getHostname() + ":" +
+            Integer.toString(sn.getPort()),
+            sn.getStartcode(), (int) load.getUsedHeapSize().get(Size.Unit.MEGABYTE),
+            (int) load.getMaxHeapSize().get(Size.Unit.MEGABYTE));
+        node.setRequests(load.getRequestCount());
+        for (RegionMetrics region: load.getRegionMetrics().values()) {
+          node.addRegion(region.getRegionName(), region.getStoreCount(),
+            region.getStoreFileCount(),
+            (int) region.getStoreFileSize().get(Size.Unit.MEGABYTE),
+            (int) region.getMemStoreSize().get(Size.Unit.MEGABYTE),
+            (long) region.getStoreFileIndexSize().get(Size.Unit.KILOBYTE),
+            region.getReadRequestCount(),
+            region.getWriteRequestCount(),
+            (int) region.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE),
+            (int) region.getStoreFileUncompressedDataIndexSize().get(Size.Unit.KILOBYTE),
+            (int) region.getBloomFilterSize().get(Size.Unit.KILOBYTE),
+            region.getCompactingCellCount(),
+            region.getCompactedCellCount());
         }
       }
       for (ServerName name: status.getDeadServerNames()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java
index 3ac6566..aa4abea 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java
@@ -64,7 +64,7 @@ public class StorageClusterVersionResource extends ResourceBase {
     try {
       StorageClusterVersionModel model = new StorageClusterVersionModel();
       model.setVersion(
-        servlet.getAdmin().getClusterStatus(EnumSet.of(Option.HBASE_VERSION))
+        servlet.getAdmin().getClusterMetrics(EnumSet.of(Option.HBASE_VERSION))
             .getHBaseVersion());
       ResponseBuilder response = Response.ok(model);
       response.cacheControl(cacheControl);

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
index c0278c6..1c70925 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
@@ -29,9 +29,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
@@ -73,7 +72,7 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
   private static final Logger LOG = LoggerFactory.getLogger(RSGroupBasedLoadBalancer.class);
 
   private Configuration config;
-  private ClusterStatus clusterStatus;
+  private ClusterMetrics clusterStatus;
   private MasterServices masterServices;
   private volatile RSGroupInfoManager rsGroupInfoManager;
   private LoadBalancer internalBalancer;
@@ -95,8 +94,8 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
   }
 
   @Override
-  public void setClusterStatus(ClusterStatus st) {
-    this.clusterStatus = st;
+  public void setClusterMetrics(ClusterMetrics sm) {
+    this.clusterStatus = sm;
   }
 
   @Override
@@ -386,7 +385,7 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
         StochasticLoadBalancer.class, LoadBalancer.class);
     internalBalancer = ReflectionUtils.newInstance(balancerKlass, config);
     internalBalancer.setMasterServices(masterServices);
-    internalBalancer.setClusterStatus(clusterStatus);
+    internalBalancer.setClusterMetrics(clusterStatus);
     internalBalancer.setConf(config);
     internalBalancer.initialize();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
index 5c8c2aa..1d28233 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
@@ -32,14 +32,15 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseCluster;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerMetrics;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
@@ -163,15 +164,16 @@ public abstract class TestRSGroupsBase {
   public Map<TableName, Map<ServerName, List<String>>> getTableServerRegionMap()
       throws IOException {
     Map<TableName, Map<ServerName, List<String>>> map = Maps.newTreeMap();
-    ClusterStatus status = TEST_UTIL.getHBaseClusterInterface().getClusterStatus();
-    for(ServerName serverName : status.getServers()) {
-      for(RegionLoad rl : status.getLoad(serverName).getRegionsLoad().values()) {
+    ClusterMetrics status = TEST_UTIL.getHBaseClusterInterface().getClusterMetrics();
+    for (Map.Entry<ServerName, ServerMetrics> entry : status.getLiveServerMetrics().entrySet()) {
+      ServerName serverName = entry.getKey();
+      for(RegionMetrics rl : entry.getValue().getRegionMetrics().values()) {
         TableName tableName = null;
         try {
-          tableName = RegionInfo.getTable(rl.getName());
+          tableName = RegionInfo.getTable(rl.getRegionName());
         } catch (IllegalArgumentException e) {
           LOG.warn("Failed parse a table name from regionname=" +
-              Bytes.toStringBinary(rl.getName()));
+            Bytes.toStringBinary(rl.getRegionName()));
           continue;
         }
         if(!map.containsKey(tableName)) {
@@ -267,11 +269,11 @@ public abstract class TestRSGroupsBase {
 
   // return the real number of region servers, excluding the master embedded region server in 2.0+
   public int getNumServers() throws IOException {
-    ClusterStatus status =
-        admin.getClusterStatus(EnumSet.of(Option.MASTER, Option.LIVE_SERVERS));
-    ServerName master = status.getMaster();
+    ClusterMetrics status =
+        admin.getClusterMetrics(EnumSet.of(Option.MASTER, Option.LIVE_SERVERS));
+    ServerName master = status.getMasterName();
     int count = 0;
-    for (ServerName sn : status.getServers()) {
+    for (ServerName sn : status.getLiveServerMetrics().keySet()) {
       if (!sn.equals(master)) {
         count++;
       }
@@ -494,8 +496,8 @@ public abstract class TestRSGroupsBase {
     }
     //get server which is not a member of new group
     ServerName targetServer = null;
-    for (ServerName server : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
-                                  .getServers()) {
+    for (ServerName server : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+                                  .getLiveServerMetrics().keySet()) {
       if (!newGroup.containsServer(server.getAddress())) {
         targetServer = server;
         break;
@@ -524,7 +526,7 @@ public abstract class TestRSGroupsBase {
         return
             getTableRegionMap().get(tableName) != null &&
                 getTableRegionMap().get(tableName).size() == 6 &&
-                admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
+                admin.getClusterMetrics(EnumSet.of(Option.REGIONS_IN_TRANSITION))
                      .getRegionStatesInTransition().size() < 1;
       }
     });
@@ -603,13 +605,13 @@ public abstract class TestRSGroupsBase {
           AdminProtos.StopServerRequest.newBuilder().setReason("Die").build());
     } catch(Exception e) {
     }
-    assertFalse(cluster.getClusterStatus().getServers().contains(targetServer));
+    assertFalse(cluster.getClusterMetrics().getLiveServerMetrics().containsKey(targetServer));
 
     //wait for created table to be assigned
     TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        return cluster.getClusterStatus().getRegionStatesInTransition().isEmpty();
+        return cluster.getClusterMetrics().getRegionStatesInTransition().isEmpty();
       }
     });
     Set<Address> newServers = Sets.newHashSet();
@@ -626,7 +628,7 @@ public abstract class TestRSGroupsBase {
     TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        return cluster.getClusterStatus().getRegionStatesInTransition().isEmpty();
+        return cluster.getClusterMetrics().getRegionStatesInTransition().isEmpty();
       }
     });
 
@@ -766,7 +768,8 @@ public abstract class TestRSGroupsBase {
 
     //get server which is not a member of new group
     ServerName targetServer = null;
-    for(ServerName server : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
+    for(ServerName server : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+      .getLiveServerMetrics().keySet()) {
       if(!newGroup.containsServer(server.getAddress()) &&
            !rsGroupAdmin.getRSGroupInfo("master").containsServer(server.getAddress())) {
         targetServer = server;
@@ -829,7 +832,7 @@ public abstract class TestRSGroupsBase {
         return getTableRegionMap().get(tableName) != null &&
                 getTableRegionMap().get(tableName).size() == 5 &&
                 getTableServerRegionMap().get(tableName).size() == 1 &&
-                admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
+                admin.getClusterMetrics(EnumSet.of(Option.REGIONS_IN_TRANSITION))
                      .getRegionStatesInTransition().size() < 1;
       }
     });
@@ -890,11 +893,11 @@ public abstract class TestRSGroupsBase {
       @Override
       public boolean evaluate() throws Exception {
         return !master.getServerManager().areDeadServersInProgress()
-            && cluster.getClusterStatus().getDeadServerNames().size() > 0;
+            && cluster.getClusterMetrics().getDeadServerNames().size() > 0;
       }
     });
-    assertFalse(cluster.getClusterStatus().getServers().contains(targetServer));
-    assertTrue(cluster.getClusterStatus().getDeadServerNames().contains(targetServer));
+    assertFalse(cluster.getClusterMetrics().getLiveServerMetrics().containsKey(targetServer));
+    assertTrue(cluster.getClusterMetrics().getDeadServerNames().contains(targetServer));
     assertTrue(newGroup.getServers().contains(targetServer.getAddress()));
 
     //clear dead servers list
@@ -941,7 +944,7 @@ public abstract class TestRSGroupsBase {
       @Override
       public boolean evaluate() throws Exception {
         return !master.getServerManager().areDeadServersInProgress()
-            && cluster.getClusterStatus().getDeadServerNames().size() > 0;
+            && cluster.getClusterMetrics().getDeadServerNames().size() > 0;
       }
     });
 
@@ -956,15 +959,15 @@ public abstract class TestRSGroupsBase {
     }
     assertTrue(newGroup.getServers().contains(targetServer.getAddress()));
 
-    ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterStatus().getMaster();
+    ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterMetrics().getMasterName();
     TEST_UTIL.getHBaseClusterInterface().stopMaster(sn);
     TEST_UTIL.getHBaseClusterInterface().waitForMasterToStop(sn, 60000);
     TEST_UTIL.getHBaseClusterInterface().startMaster(sn.getHostname(), 0);
     TEST_UTIL.getHBaseClusterInterface().waitForActiveAndReadyMaster(60000);
 
-    assertEquals(3, cluster.getClusterStatus().getServersSize());
-    assertFalse(cluster.getClusterStatus().getServers().contains(targetServer));
-    assertFalse(cluster.getClusterStatus().getDeadServerNames().contains(targetServer));
+    assertEquals(3, cluster.getClusterMetrics().getLiveServerMetrics().size());
+    assertFalse(cluster.getClusterMetrics().getLiveServerMetrics().containsKey(targetServer));
+    assertFalse(cluster.getClusterMetrics().getDeadServerNames().contains(targetServer));
     assertTrue(newGroup.getServers().contains(targetServer.getAddress()));
 
     rsGroupAdmin.removeServers(Sets.newHashSet(targetServer.getAddress()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/BackupMasterStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/BackupMasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/BackupMasterStatusTmpl.jamon
index e1f864e..a49a5fa 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/BackupMasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/BackupMasterStatusTmpl.jamon
@@ -54,8 +54,8 @@ MasterAddressTracker masterAddressTracker = master.getMasterAddressTracker();
         <th>Start Time</th>
     </tr>
     <%java>
-    Collection<ServerName> backup_masters = master.getClusterStatusWithoutCoprocessor(
-        EnumSet.of(ClusterMetrics.Option.BACKUP_MASTERS)).getBackupMasters();
+    Collection<ServerName> backup_masters = master.getClusterMetricsWithoutCoprocessor(
+        EnumSet.of(ClusterMetrics.Option.BACKUP_MASTERS)).getBackupMasterNames();
     ServerName [] backupServerNames = backup_masters.toArray(new ServerName[backup_masters.size()]);
     Arrays.sort(backupServerNames);
     for (ServerName serverName : backupServerNames) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index 6ef5504..a17bc9f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -21,8 +21,7 @@ package org.apache.hadoop.hbase.coprocessor;
 import java.io.IOException;
 import java.util.List;
 import java.util.Set;
-
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.MetaMutationAnnotation;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -1263,14 +1262,14 @@ public interface MasterObserver {
   /**
    * Called before get cluster status.
    */
-  default void preGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx)
+  default void preGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx)
       throws IOException {}
 
   /**
    * Called after get cluster status.
    */
-  default void postGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      ClusterStatus status) throws IOException {}
+  default void postGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx,
+    ClusterMetrics status) throws IOException {}
 
   /**
    * Called before clear dead region servers.

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
index 85493d1..6b87194 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
@@ -35,8 +35,8 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ScheduledChore;
@@ -65,7 +65,6 @@ import org.apache.hbase.thirdparty.io.netty.channel.socket.InternetProtocolFamil
 import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioDatagramChannel;
 import org.apache.hbase.thirdparty.io.netty.handler.codec.MessageToMessageEncoder;
 import org.apache.hbase.thirdparty.io.netty.util.internal.StringUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 
 
 /**
@@ -159,12 +158,12 @@ public class ClusterStatusPublisher extends ScheduledChore {
     // We're reusing an existing protobuf message, but we don't send everything.
     // This could be extended in the future, for example if we want to send stuff like the
     //  hbase:meta server name.
-    publisher.publish(new ClusterStatus(ClusterMetricsBuilder.newBuilder()
-        .setHBaseVersion(VersionInfo.getVersion())
-        .setClusterId(master.getMasterFileSystem().getClusterId().toString())
-        .setMasterName(master.getServerName())
-        .setDeadServerNames(sns)
-        .build()));
+    publisher.publish(ClusterMetricsBuilder.newBuilder()
+      .setHBaseVersion(VersionInfo.getVersion())
+      .setClusterId(master.getMasterFileSystem().getClusterId().toString())
+      .setMasterName(master.getServerName())
+      .setDeadServerNames(sns)
+      .build());
   }
 
   protected void cleanup() {
@@ -229,7 +228,7 @@ public class ClusterStatusPublisher extends ScheduledChore {
 
     void connect(Configuration conf) throws IOException;
 
-    void publish(ClusterStatus cs);
+    void publish(ClusterMetrics cs);
 
     @Override
     void close();
@@ -289,7 +288,7 @@ public class ClusterStatusPublisher extends ScheduledChore {
       b.group(group)
         .channelFactory(new HBaseDatagramChannelFactory<Channel>(NioDatagramChannel.class, family))
         .option(ChannelOption.SO_REUSEADDR, true)
-        .handler(new ClusterStatusEncoder(isa));
+        .handler(new ClusterMetricsEncoder(isa));
 
       try {
         channel = (DatagramChannel) b.bind(bindAddress, 0).sync().channel();
@@ -328,24 +327,24 @@ public class ClusterStatusPublisher extends ScheduledChore {
       }
     }
 
-    private static final class ClusterStatusEncoder extends MessageToMessageEncoder<ClusterStatus> {
+    private static final class ClusterMetricsEncoder
+        extends MessageToMessageEncoder<ClusterMetrics> {
       final private InetSocketAddress isa;
 
-      private ClusterStatusEncoder(InetSocketAddress isa) {
+      private ClusterMetricsEncoder(InetSocketAddress isa) {
         this.isa = isa;
       }
 
       @Override
       protected void encode(ChannelHandlerContext channelHandlerContext,
-                            ClusterStatus clusterStatus, List<Object> objects) {
-        ClusterStatusProtos.ClusterStatus csp
-          = ClusterMetricsBuilder.toClusterStatus(clusterStatus);
-        objects.add(new DatagramPacket(Unpooled.wrappedBuffer(csp.toByteArray()), isa));
+        ClusterMetrics clusterStatus, List<Object> objects) {
+        objects.add(new DatagramPacket(Unpooled.wrappedBuffer(
+          ClusterMetricsBuilder.toClusterStatus(clusterStatus).toByteArray()), isa));
       }
     }
 
     @Override
-    public void publish(ClusterStatus cs) {
+    public void publish(ClusterMetrics cs) {
       channel.writeAndFlush(cs).syncUninterruptibly();
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/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 dec1040..945f54d 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
@@ -56,9 +56,9 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
@@ -834,7 +834,7 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     //initialize load balancer
     this.balancer.setMasterServices(this);
-    this.balancer.setClusterStatus(getClusterStatusWithoutCoprocessor());
+    this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
     this.balancer.initialize();
 
     // Check if master is shutting down because of some issue
@@ -878,7 +878,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.assignmentManager.joinCluster();
 
     // set cluster status again after user regions are assigned
-    this.balancer.setClusterStatus(getClusterStatusWithoutCoprocessor());
+    this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
 
     // Start balancer and meta catalog janitor after meta and regions have been assigned.
     status.setStatus("Starting balancer and catalog janitor");
@@ -1404,7 +1404,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       List<RegionPlan> plans = new ArrayList<>();
 
       //Give the balancer the current cluster state.
-      this.balancer.setClusterStatus(getClusterStatusWithoutCoprocessor());
+      this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
       this.balancer.setClusterLoad(assignmentsByTable);
 
       for (Map<ServerName, List<RegionInfo>> serverMap : assignmentsByTable.values()) {
@@ -2404,11 +2404,11 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
-  public ClusterStatus getClusterStatusWithoutCoprocessor() throws InterruptedIOException {
-    return getClusterStatusWithoutCoprocessor(EnumSet.allOf(Option.class));
+  public ClusterMetrics getClusterMetricsWithoutCoprocessor() throws InterruptedIOException {
+    return getClusterMetricsWithoutCoprocessor(EnumSet.allOf(Option.class));
   }
 
-  public ClusterStatus getClusterStatusWithoutCoprocessor(EnumSet<Option> options)
+  public ClusterMetrics getClusterMetricsWithoutCoprocessor(EnumSet<Option> options)
       throws InterruptedIOException {
     ClusterMetricsBuilder builder = ClusterMetricsBuilder.newBuilder();
     // given that hbase1 can't submit the request with Option,
@@ -2464,23 +2464,23 @@ public class HMaster extends HRegionServer implements MasterServices {
         }
       }
     }
-    return new ClusterStatus(builder.build());
+    return builder.build();
   }
 
   /**
    * @return cluster status
    */
-  public ClusterStatus getClusterStatus() throws IOException {
-    return getClusterStatus(EnumSet.allOf(Option.class));
+  public ClusterMetrics getClusterMetrics() throws IOException {
+    return getClusterMetrics(EnumSet.allOf(Option.class));
   }
 
-  public ClusterStatus getClusterStatus(EnumSet<Option> options) throws IOException {
+  public ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException {
     if (cpHost != null) {
-      cpHost.preGetClusterStatus();
+      cpHost.preGetClusterMetrics();
     }
-    ClusterStatus status = getClusterStatusWithoutCoprocessor(options);
+    ClusterMetrics status = getClusterMetricsWithoutCoprocessor(options);
     if (cpHost != null) {
-      cpHost.postGetClusterStatus(status);
+      cpHost.postGetClusterMetrics(status);
     }
     return status;
   }
@@ -3173,14 +3173,14 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   @Override
   public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
-    return getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
-        .getLastMajorCompactionTsForTable(table);
+    return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+        .getLastMajorCompactionTimestamp(table);
   }
 
   @Override
   public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
-    return getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
-        .getLastMajorCompactionTsForRegion(regionName);
+    return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+        .getLastMajorCompactionTimestamp(regionName);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/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 1dad70d..f1a0593 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
@@ -18,12 +18,12 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import edu.umd.cs.findbugs.annotations.Nullable;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Stoppable;
@@ -32,8 +32,6 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import edu.umd.cs.findbugs.annotations.Nullable;
-
 /**
  * Makes decisions about the placement and movement of Regions across
  * RegionServers.
@@ -71,7 +69,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
    * Set the current cluster status.  This allows a LoadBalancer to map host name to a server
    * @param st
    */
-  void setClusterStatus(ClusterStatus st);
+  void setClusterMetrics(ClusterMetrics st);
 
   /**
    * Pass RegionStates and allow balancer to set the current cluster load.

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 9eb2ec8..10e1d0a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -18,14 +18,12 @@
 
 package org.apache.hadoop.hbase.master;
 
+import com.google.protobuf.Service;
 import java.io.IOException;
 import java.util.List;
 import java.util.Set;
-
-import com.google.protobuf.Service;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.MetaMutationAnnotation;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
@@ -1563,20 +1561,20 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preGetClusterStatus() throws IOException {
+  public void preGetClusterMetrics() throws IOException {
     execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
       public void call(MasterObserver observer) throws IOException {
-        observer.preGetClusterStatus(this);
+        observer.preGetClusterMetrics(this);
       }
     });
   }
 
-  public void postGetClusterStatus(ClusterStatus status) throws IOException {
+  public void postGetClusterMetrics(ClusterMetrics status) throws IOException {
     execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
       public void call(MasterObserver observer) throws IOException {
-        observer.postGetClusterStatus(this, status);
+        observer.postGetClusterMetrics(this, status);
       }
     });
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 5ada8fd..8f41e4f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -30,7 +30,6 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -905,7 +904,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       master.checkInitialized();
       response.setClusterStatus(ClusterMetricsBuilder.toClusterStatus(
-        master.getClusterStatus(ClusterMetricsBuilder.toOptions(req.getOptionsList()))));
+        master.getClusterMetrics(ClusterMetricsBuilder.toOptions(req.getOptionsList()))));
     } catch (IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
index 96b31c7..68ec524 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
@@ -191,265 +191,268 @@ public class RegionPlacementMaintainer {
    * @throws IOException
    */
   private void genAssignmentPlan(TableName tableName,
-      SnapshotOfRegionAssignmentFromMeta assignmentSnapshot,
-      Map<String, Map<String, Float>> regionLocalityMap, FavoredNodesPlan plan,
-      boolean munkresForSecondaryAndTertiary) throws IOException {
-      // Get the all the regions for the current table
-      List<RegionInfo> regions =
-        assignmentSnapshot.getTableToRegionMap().get(tableName);
-      int numRegions = regions.size();
-
-      // Get the current assignment map
-      Map<RegionInfo, ServerName> currentAssignmentMap =
-        assignmentSnapshot.getRegionToRegionServerMap();
-
-      // Get the all the region servers
-      List<ServerName> servers = new ArrayList<>();
-      try (Admin admin = this.connection.getAdmin()) {
-        servers.addAll(admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers());
-      }
-
-      LOG.info("Start to generate assignment plan for " + numRegions +
-          " regions from table " + tableName + " with " +
-          servers.size() + " region servers");
+    SnapshotOfRegionAssignmentFromMeta assignmentSnapshot,
+    Map<String, Map<String, Float>> regionLocalityMap, FavoredNodesPlan plan,
+    boolean munkresForSecondaryAndTertiary) throws IOException {
+    // Get the all the regions for the current table
+    List<RegionInfo> regions =
+      assignmentSnapshot.getTableToRegionMap().get(tableName);
+    int numRegions = regions.size();
+
+    // Get the current assignment map
+    Map<RegionInfo, ServerName> currentAssignmentMap =
+      assignmentSnapshot.getRegionToRegionServerMap();
+
+    // Get the all the region servers
+    List<ServerName> servers = new ArrayList<>();
+    try (Admin admin = this.connection.getAdmin()) {
+      servers.addAll(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+        .getLiveServerMetrics().keySet());
+    }
 
-      int slotsPerServer = (int) Math.ceil((float) numRegions /
-          servers.size());
-      int regionSlots = slotsPerServer * servers.size();
-
-      // Compute the primary, secondary and tertiary costs for each region/server
-      // pair. These costs are based only on node locality and rack locality, and
-      // will be modified later.
-      float[][] primaryCost = new float[numRegions][regionSlots];
-      float[][] secondaryCost = new float[numRegions][regionSlots];
-      float[][] tertiaryCost = new float[numRegions][regionSlots];
-
-      if (this.enforceLocality && regionLocalityMap != null) {
-        // Transform the locality mapping into a 2D array, assuming that any
-        // unspecified locality value is 0.
-        float[][] localityPerServer = new float[numRegions][regionSlots];
-        for (int i = 0; i < numRegions; i++) {
-          Map<String, Float> serverLocalityMap =
-              regionLocalityMap.get(regions.get(i).getEncodedName());
-          if (serverLocalityMap == null) {
+    LOG.info("Start to generate assignment plan for " + numRegions +
+        " regions from table " + tableName + " with " +
+        servers.size() + " region servers");
+
+    int slotsPerServer = (int) Math.ceil((float) numRegions /
+        servers.size());
+    int regionSlots = slotsPerServer * servers.size();
+
+    // Compute the primary, secondary and tertiary costs for each region/server
+    // pair. These costs are based only on node locality and rack locality, and
+    // will be modified later.
+    float[][] primaryCost = new float[numRegions][regionSlots];
+    float[][] secondaryCost = new float[numRegions][regionSlots];
+    float[][] tertiaryCost = new float[numRegions][regionSlots];
+
+    if (this.enforceLocality && regionLocalityMap != null) {
+      // Transform the locality mapping into a 2D array, assuming that any
+      // unspecified locality value is 0.
+      float[][] localityPerServer = new float[numRegions][regionSlots];
+      for (int i = 0; i < numRegions; i++) {
+        Map<String, Float> serverLocalityMap =
+            regionLocalityMap.get(regions.get(i).getEncodedName());
+        if (serverLocalityMap == null) {
+          continue;
+        }
+        for (int j = 0; j < servers.size(); j++) {
+          String serverName = servers.get(j).getHostname();
+          if (serverName == null) {
             continue;
           }
-          for (int j = 0; j < servers.size(); j++) {
-            String serverName = servers.get(j).getHostname();
-            if (serverName == null) {
-              continue;
-            }
-            Float locality = serverLocalityMap.get(serverName);
-            if (locality == null) {
-              continue;
-            }
-            for (int k = 0; k < slotsPerServer; k++) {
-              // If we can't find the locality of a region to a server, which occurs
-              // because locality is only reported for servers which have some
-              // blocks of a region local, then the locality for that pair is 0.
-              localityPerServer[i][j * slotsPerServer + k] = locality.floatValue();
-            }
+          Float locality = serverLocalityMap.get(serverName);
+          if (locality == null) {
+            continue;
+          }
+          for (int k = 0; k < slotsPerServer; k++) {
+            // If we can't find the locality of a region to a server, which occurs
+            // because locality is only reported for servers which have some
+            // blocks of a region local, then the locality for that pair is 0.
+            localityPerServer[i][j * slotsPerServer + k] = locality.floatValue();
           }
         }
+      }
 
-        // Compute the total rack locality for each region in each rack. The total
-        // rack locality is the sum of the localities of a region on all servers in
-        // a rack.
-        Map<String, Map<RegionInfo, Float>> rackRegionLocality = new HashMap<>();
-        for (int i = 0; i < numRegions; i++) {
-          RegionInfo region = regions.get(i);
-          for (int j = 0; j < regionSlots; j += slotsPerServer) {
-            String rack = rackManager.getRack(servers.get(j / slotsPerServer));
-            Map<RegionInfo, Float> rackLocality = rackRegionLocality.get(rack);
-            if (rackLocality == null) {
-              rackLocality = new HashMap<>();
-              rackRegionLocality.put(rack, rackLocality);
-            }
-            Float localityObj = rackLocality.get(region);
-            float locality = localityObj == null ? 0 : localityObj.floatValue();
-            locality += localityPerServer[i][j];
-            rackLocality.put(region, locality);
+      // Compute the total rack locality for each region in each rack. The total
+      // rack locality is the sum of the localities of a region on all servers in
+      // a rack.
+      Map<String, Map<RegionInfo, Float>> rackRegionLocality = new HashMap<>();
+      for (int i = 0; i < numRegions; i++) {
+        RegionInfo region = regions.get(i);
+        for (int j = 0; j < regionSlots; j += slotsPerServer) {
+          String rack = rackManager.getRack(servers.get(j / slotsPerServer));
+          Map<RegionInfo, Float> rackLocality = rackRegionLocality.get(rack);
+          if (rackLocality == null) {
+            rackLocality = new HashMap<>();
+            rackRegionLocality.put(rack, rackLocality);
           }
+          Float localityObj = rackLocality.get(region);
+          float locality = localityObj == null ? 0 : localityObj.floatValue();
+          locality += localityPerServer[i][j];
+          rackLocality.put(region, locality);
         }
-        for (int i = 0; i < numRegions; i++) {
-          for (int j = 0; j < regionSlots; j++) {
-            String rack = rackManager.getRack(servers.get(j / slotsPerServer));
-            Float totalRackLocalityObj =
-                rackRegionLocality.get(rack).get(regions.get(i));
-            float totalRackLocality = totalRackLocalityObj == null ?
-                0 : totalRackLocalityObj.floatValue();
-
-            // Primary cost aims to favor servers with high node locality and low
-            // rack locality, so that secondaries and tertiaries can be chosen for
-            // nodes with high rack locality. This might give primaries with
-            // slightly less locality at first compared to a cost which only
-            // considers the node locality, but should be better in the long run.
-            primaryCost[i][j] = 1 - (2 * localityPerServer[i][j] -
-                totalRackLocality);
-
-            // Secondary cost aims to favor servers with high node locality and high
-            // rack locality since the tertiary will be chosen from the same rack as
-            // the secondary. This could be negative, but that is okay.
-            secondaryCost[i][j] = 2 - (localityPerServer[i][j] + totalRackLocality);
-
-            // Tertiary cost is only concerned with the node locality. It will later
-            // be restricted to only hosts on the same rack as the secondary.
-            tertiaryCost[i][j] = 1 - localityPerServer[i][j];
-          }
+      }
+      for (int i = 0; i < numRegions; i++) {
+        for (int j = 0; j < regionSlots; j++) {
+          String rack = rackManager.getRack(servers.get(j / slotsPerServer));
+          Float totalRackLocalityObj =
+              rackRegionLocality.get(rack).get(regions.get(i));
+          float totalRackLocality = totalRackLocalityObj == null ?
+              0 : totalRackLocalityObj.floatValue();
+
+          // Primary cost aims to favor servers with high node locality and low
+          // rack locality, so that secondaries and tertiaries can be chosen for
+          // nodes with high rack locality. This might give primaries with
+          // slightly less locality at first compared to a cost which only
+          // considers the node locality, but should be better in the long run.
+          primaryCost[i][j] = 1 - (2 * localityPerServer[i][j] -
+              totalRackLocality);
+
+          // Secondary cost aims to favor servers with high node locality and high
+          // rack locality since the tertiary will be chosen from the same rack as
+          // the secondary. This could be negative, but that is okay.
+          secondaryCost[i][j] = 2 - (localityPerServer[i][j] + totalRackLocality);
+
+          // Tertiary cost is only concerned with the node locality. It will later
+          // be restricted to only hosts on the same rack as the secondary.
+          tertiaryCost[i][j] = 1 - localityPerServer[i][j];
         }
       }
+    }
 
-      if (this.enforceMinAssignmentMove && currentAssignmentMap != null) {
-        // We want to minimize the number of regions which move as the result of a
-        // new assignment. Therefore, slightly penalize any placement which is for
-        // a host that is not currently serving the region.
-        for (int i = 0; i < numRegions; i++) {
-          for (int j = 0; j < servers.size(); j++) {
-            ServerName currentAddress = currentAssignmentMap.get(regions.get(i));
-            if (currentAddress != null &&
-                !currentAddress.equals(servers.get(j))) {
-              for (int k = 0; k < slotsPerServer; k++) {
-                primaryCost[i][j * slotsPerServer + k] += NOT_CURRENT_HOST_PENALTY;
-              }
+    if (this.enforceMinAssignmentMove && currentAssignmentMap != null) {
+      // We want to minimize the number of regions which move as the result of a
+      // new assignment. Therefore, slightly penalize any placement which is for
+      // a host that is not currently serving the region.
+      for (int i = 0; i < numRegions; i++) {
+        for (int j = 0; j < servers.size(); j++) {
+          ServerName currentAddress = currentAssignmentMap.get(regions.get(i));
+          if (currentAddress != null &&
+              !currentAddress.equals(servers.get(j))) {
+            for (int k = 0; k < slotsPerServer; k++) {
+              primaryCost[i][j * slotsPerServer + k] += NOT_CURRENT_HOST_PENALTY;
             }
           }
         }
       }
+    }
 
-      // Artificially increase cost of last slot of each server to evenly
-      // distribute the slop, otherwise there will be a few servers with too few
-      // regions and many servers with the max number of regions.
-      for (int i = 0; i < numRegions; i++) {
-        for (int j = 0; j < regionSlots; j += slotsPerServer) {
-          primaryCost[i][j] += LAST_SLOT_COST_PENALTY;
-          secondaryCost[i][j] += LAST_SLOT_COST_PENALTY;
-          tertiaryCost[i][j] += LAST_SLOT_COST_PENALTY;
-        }
+    // Artificially increase cost of last slot of each server to evenly
+    // distribute the slop, otherwise there will be a few servers with too few
+    // regions and many servers with the max number of regions.
+    for (int i = 0; i < numRegions; i++) {
+      for (int j = 0; j < regionSlots; j += slotsPerServer) {
+        primaryCost[i][j] += LAST_SLOT_COST_PENALTY;
+        secondaryCost[i][j] += LAST_SLOT_COST_PENALTY;
+        tertiaryCost[i][j] += LAST_SLOT_COST_PENALTY;
       }
+    }
 
-      RandomizedMatrix randomizedMatrix = new RandomizedMatrix(numRegions,
-          regionSlots);
-      primaryCost = randomizedMatrix.transform(primaryCost);
-      int[] primaryAssignment = new MunkresAssignment(primaryCost).solve();
-      primaryAssignment = randomizedMatrix.invertIndices(primaryAssignment);
-
-      // Modify the secondary and tertiary costs for each region/server pair to
-      // prevent a region from being assigned to the same rack for both primary
-      // and either one of secondary or tertiary.
+    RandomizedMatrix randomizedMatrix = new RandomizedMatrix(numRegions,
+        regionSlots);
+    primaryCost = randomizedMatrix.transform(primaryCost);
+    int[] primaryAssignment = new MunkresAssignment(primaryCost).solve();
+    primaryAssignment = randomizedMatrix.invertIndices(primaryAssignment);
+
+    // Modify the secondary and tertiary costs for each region/server pair to
+    // prevent a region from being assigned to the same rack for both primary
+    // and either one of secondary or tertiary.
+    for (int i = 0; i < numRegions; i++) {
+      int slot = primaryAssignment[i];
+      String rack = rackManager.getRack(servers.get(slot / slotsPerServer));
+      for (int k = 0; k < servers.size(); k++) {
+        if (!rackManager.getRack(servers.get(k)).equals(rack)) {
+          continue;
+        }
+        if (k == slot / slotsPerServer) {
+          // Same node, do not place secondary or tertiary here ever.
+          for (int m = 0; m < slotsPerServer; m++) {
+            secondaryCost[i][k * slotsPerServer + m] = MAX_COST;
+            tertiaryCost[i][k * slotsPerServer + m] = MAX_COST;
+          }
+        } else {
+          // Same rack, do not place secondary or tertiary here if possible.
+          for (int m = 0; m < slotsPerServer; m++) {
+            secondaryCost[i][k * slotsPerServer + m] = AVOID_COST;
+            tertiaryCost[i][k * slotsPerServer + m] = AVOID_COST;
+          }
+        }
+      }
+    }
+    if (munkresForSecondaryAndTertiary) {
+      randomizedMatrix = new RandomizedMatrix(numRegions, regionSlots);
+      secondaryCost = randomizedMatrix.transform(secondaryCost);
+      int[] secondaryAssignment = new MunkresAssignment(secondaryCost).solve();
+      secondaryAssignment = randomizedMatrix.invertIndices(secondaryAssignment);
+
+      // Modify the tertiary costs for each region/server pair to ensure that a
+      // region is assigned to a tertiary server on the same rack as its secondary
+      // server, but not the same server in that rack.
       for (int i = 0; i < numRegions; i++) {
-        int slot = primaryAssignment[i];
+        int slot = secondaryAssignment[i];
         String rack = rackManager.getRack(servers.get(slot / slotsPerServer));
         for (int k = 0; k < servers.size(); k++) {
-          if (!rackManager.getRack(servers.get(k)).equals(rack)) {
-            continue;
-          }
           if (k == slot / slotsPerServer) {
-            // Same node, do not place secondary or tertiary here ever.
+            // Same node, do not place tertiary here ever.
             for (int m = 0; m < slotsPerServer; m++) {
-              secondaryCost[i][k * slotsPerServer + m] = MAX_COST;
               tertiaryCost[i][k * slotsPerServer + m] = MAX_COST;
             }
           } else {
-            // Same rack, do not place secondary or tertiary here if possible.
+            if (rackManager.getRack(servers.get(k)).equals(rack)) {
+              continue;
+            }
+            // Different rack, do not place tertiary here if possible.
             for (int m = 0; m < slotsPerServer; m++) {
-              secondaryCost[i][k * slotsPerServer + m] = AVOID_COST;
               tertiaryCost[i][k * slotsPerServer + m] = AVOID_COST;
             }
           }
         }
       }
-      if (munkresForSecondaryAndTertiary) {
-        randomizedMatrix = new RandomizedMatrix(numRegions, regionSlots);
-        secondaryCost = randomizedMatrix.transform(secondaryCost);
-        int[] secondaryAssignment = new MunkresAssignment(secondaryCost).solve();
-        secondaryAssignment = randomizedMatrix.invertIndices(secondaryAssignment);
-
-        // Modify the tertiary costs for each region/server pair to ensure that a
-        // region is assigned to a tertiary server on the same rack as its secondary
-        // server, but not the same server in that rack.
-        for (int i = 0; i < numRegions; i++) {
-          int slot = secondaryAssignment[i];
-          String rack = rackManager.getRack(servers.get(slot / slotsPerServer));
-          for (int k = 0; k < servers.size(); k++) {
-            if (k == slot / slotsPerServer) {
-              // Same node, do not place tertiary here ever.
-              for (int m = 0; m < slotsPerServer; m++) {
-                tertiaryCost[i][k * slotsPerServer + m] = MAX_COST;
-              }
-            } else {
-              if (rackManager.getRack(servers.get(k)).equals(rack)) {
-                continue;
-              }
-              // Different rack, do not place tertiary here if possible.
-              for (int m = 0; m < slotsPerServer; m++) {
-                tertiaryCost[i][k * slotsPerServer + m] = AVOID_COST;
-              }
-            }
-          }
-        }
 
-        randomizedMatrix = new RandomizedMatrix(numRegions, regionSlots);
-        tertiaryCost = randomizedMatrix.transform(tertiaryCost);
-        int[] tertiaryAssignment = new MunkresAssignment(tertiaryCost).solve();
-        tertiaryAssignment = randomizedMatrix.invertIndices(tertiaryAssignment);
-
-        for (int i = 0; i < numRegions; i++) {
-          List<ServerName> favoredServers = new ArrayList<>(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
-          ServerName s = servers.get(primaryAssignment[i] / slotsPerServer);
-          favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
-              ServerName.NON_STARTCODE));
-
-          s = servers.get(secondaryAssignment[i] / slotsPerServer);
-          favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
-              ServerName.NON_STARTCODE));
-
-          s = servers.get(tertiaryAssignment[i] / slotsPerServer);
-          favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
-              ServerName.NON_STARTCODE));
-          // Update the assignment plan
-          plan.updateFavoredNodesMap(regions.get(i), favoredServers);
-        }
-        LOG.info("Generated the assignment plan for " + numRegions +
-            " regions from table " + tableName + " with " +
-            servers.size() + " region servers");
-        LOG.info("Assignment plan for secondary and tertiary generated " +
-            "using MunkresAssignment");
-      } else {
-        Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>();
-        for (int i = 0; i < numRegions; i++) {
-          primaryRSMap.put(regions.get(i), servers.get(primaryAssignment[i] / slotsPerServer));
-        }
-        FavoredNodeAssignmentHelper favoredNodeHelper =
-            new FavoredNodeAssignmentHelper(servers, conf);
-        favoredNodeHelper.initialize();
-        Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap =
-            favoredNodeHelper.placeSecondaryAndTertiaryWithRestrictions(primaryRSMap);
-        for (int i = 0; i < numRegions; i++) {
-          List<ServerName> favoredServers = new ArrayList<>(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
-          RegionInfo currentRegion = regions.get(i);
-          ServerName s = primaryRSMap.get(currentRegion);
-          favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
-              ServerName.NON_STARTCODE));
-
-          ServerName[] secondaryAndTertiary =
-              secondaryAndTertiaryMap.get(currentRegion);
-          s = secondaryAndTertiary[0];
-          favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
-              ServerName.NON_STARTCODE));
-
-          s = secondaryAndTertiary[1];
-          favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
-              ServerName.NON_STARTCODE));
-          // Update the assignment plan
-          plan.updateFavoredNodesMap(regions.get(i), favoredServers);
-        }
-        LOG.info("Generated the assignment plan for " + numRegions +
-            " regions from table " + tableName + " with " +
-            servers.size() + " region servers");
-        LOG.info("Assignment plan for secondary and tertiary generated " +
-            "using placeSecondaryAndTertiaryWithRestrictions method");
+      randomizedMatrix = new RandomizedMatrix(numRegions, regionSlots);
+      tertiaryCost = randomizedMatrix.transform(tertiaryCost);
+      int[] tertiaryAssignment = new MunkresAssignment(tertiaryCost).solve();
+      tertiaryAssignment = randomizedMatrix.invertIndices(tertiaryAssignment);
+
+      for (int i = 0; i < numRegions; i++) {
+        List<ServerName> favoredServers
+          = new ArrayList<>(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
+        ServerName s = servers.get(primaryAssignment[i] / slotsPerServer);
+        favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
+            ServerName.NON_STARTCODE));
+
+        s = servers.get(secondaryAssignment[i] / slotsPerServer);
+        favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
+            ServerName.NON_STARTCODE));
+
+        s = servers.get(tertiaryAssignment[i] / slotsPerServer);
+        favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
+            ServerName.NON_STARTCODE));
+        // Update the assignment plan
+        plan.updateFavoredNodesMap(regions.get(i), favoredServers);
+      }
+      LOG.info("Generated the assignment plan for " + numRegions +
+          " regions from table " + tableName + " with " +
+          servers.size() + " region servers");
+      LOG.info("Assignment plan for secondary and tertiary generated " +
+          "using MunkresAssignment");
+    } else {
+      Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>();
+      for (int i = 0; i < numRegions; i++) {
+        primaryRSMap.put(regions.get(i), servers.get(primaryAssignment[i] / slotsPerServer));
+      }
+      FavoredNodeAssignmentHelper favoredNodeHelper =
+          new FavoredNodeAssignmentHelper(servers, conf);
+      favoredNodeHelper.initialize();
+      Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap =
+          favoredNodeHelper.placeSecondaryAndTertiaryWithRestrictions(primaryRSMap);
+      for (int i = 0; i < numRegions; i++) {
+        List<ServerName> favoredServers
+          = new ArrayList<>(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
+        RegionInfo currentRegion = regions.get(i);
+        ServerName s = primaryRSMap.get(currentRegion);
+        favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
+            ServerName.NON_STARTCODE));
+
+        ServerName[] secondaryAndTertiary =
+            secondaryAndTertiaryMap.get(currentRegion);
+        s = secondaryAndTertiary[0];
+        favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
+            ServerName.NON_STARTCODE));
+
+        s = secondaryAndTertiary[1];
+        favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
+            ServerName.NON_STARTCODE));
+        // Update the assignment plan
+        plan.updateFavoredNodesMap(regions.get(i), favoredServers);
       }
+      LOG.info("Generated the assignment plan for " + numRegions +
+          " regions from table " + tableName + " with " +
+          servers.size() + " region servers");
+      LOG.info("Assignment plan for secondary and tertiary generated " +
+          "using placeSecondaryAndTertiaryWithRestrictions method");
     }
+  }
 
   public FavoredNodesPlan getNewAssignmentPlan() throws IOException {
     // Get the current region assignment snapshot by scanning from the META

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java
index 78c460b..843a36e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java
@@ -18,13 +18,16 @@
 
 package org.apache.hadoop.hbase.master.balancer;
 
-import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.Size;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
 /**
  * Wrapper class for the few fields required by the {@link StochasticLoadBalancer}
- * from the full {@link RegionLoad}.
+ * from the full {@link RegionMetrics}.
  */
+@InterfaceAudience.Private
 @InterfaceStability.Evolving
 class BalancerRegionLoad {
   private final long readRequestsCount;
@@ -32,11 +35,11 @@ class BalancerRegionLoad {
   private final int memStoreSizeMB;
   private final int storefileSizeMB;
 
-  BalancerRegionLoad(RegionLoad regionLoad) {
-    readRequestsCount = regionLoad.getReadRequestsCount();
-    writeRequestsCount = regionLoad.getWriteRequestsCount();
-    memStoreSizeMB = regionLoad.getMemStoreSizeMB();
-    storefileSizeMB = regionLoad.getStorefileSizeMB();
+  BalancerRegionLoad(RegionMetrics regionMetrics) {
+    readRequestsCount = regionMetrics.getReadRequestCount();
+    writeRequestsCount = regionMetrics.getWriteRequestCount();
+    memStoreSizeMB = (int) regionMetrics.getMemStoreSize().get(Size.Unit.MEGABYTE);
+    storefileSizeMB = (int) regionMetrics.getStoreFileSize().get(Size.Unit.MEGABYTE);
   }
 
   public long getReadRequestsCount() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/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 750458b..a8dd9ae 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
@@ -36,7 +36,7 @@ import java.util.stream.Collectors;
 
 import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
@@ -1006,7 +1006,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   private static final Random RANDOM = new Random(System.currentTimeMillis());
   private static final Logger LOG = LoggerFactory.getLogger(BaseLoadBalancer.class);
   protected MetricsBalancer metricsBalancer = null;
-  protected ClusterStatus clusterStatus = null;
+  protected ClusterMetrics clusterStatus = null;
   protected ServerName masterServerName;
   protected MasterServices services;
   protected boolean tablesOnMaster;
@@ -1128,10 +1128,10 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   }
 
   @Override
-  public synchronized void setClusterStatus(ClusterStatus st) {
+  public synchronized void setClusterMetrics(ClusterMetrics st) {
     this.clusterStatus = st;
     if (useRegionFinder) {
-      regionFinder.setClusterStatus(st);
+      regionFinder.setClusterMetrics(st);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterStatusChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterStatusChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterStatusChore.java
index 427322d..c0383fa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterStatusChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterStatusChore.java
@@ -46,7 +46,7 @@ public class ClusterStatusChore extends ScheduledChore {
   @Override
   protected void chore() {
     try {
-      balancer.setClusterStatus(master.getClusterStatusWithoutCoprocessor());
+      balancer.setClusterMetrics(master.getClusterMetricsWithoutCoprocessor());
     } catch (InterruptedIOException e) {
       LOG.warn("Ignoring interruption", e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
index e1d878d..a9b1bb7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
@@ -27,9 +27,8 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -64,7 +63,7 @@ class RegionLocationFinder {
   private static final long CACHE_TIME = 240 * 60 * 1000;
   private static final HDFSBlocksDistribution EMPTY_BLOCK_DISTRIBUTION = new HDFSBlocksDistribution();
   private Configuration conf;
-  private volatile ClusterStatus status;
+  private volatile ClusterMetrics status;
   private MasterServices services;
   private final ListeningExecutorService executor;
   // Do not scheduleFullRefresh at master startup
@@ -105,7 +104,6 @@ class RegionLocationFinder {
 
   /**
    * Create a cache for region to list of servers
-   * @param time time to cache the locations
    * @return A new Cache.
    */
   private LoadingCache<RegionInfo, HDFSBlocksDistribution> createCache() {
@@ -126,7 +124,7 @@ class RegionLocationFinder {
     this.services = services;
   }
 
-  public void setClusterStatus(ClusterStatus status) {
+  public void setClusterMetrics(ClusterMetrics status) {
     long currentTime = EnvironmentEdgeManager.currentTime();
     this.status = status;
     if (currentTime > lastFullRefresh + (CACHE_TIME / 2)) {
@@ -244,7 +242,7 @@ class RegionLocationFinder {
     }
 
     List<ServerName> topServerNames = new ArrayList<>();
-    Collection<ServerName> regionServers = status.getServers();
+    Collection<ServerName> regionServers = status.getLiveServerMetrics().keySet();
 
     // create a mapping from hostname to ServerName for fast lookup
     HashMap<String, List<ServerName>> hostToServerName = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/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 693d8b2..6b4f943 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
@@ -27,14 +27,13 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Random;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.RegionLoad;
-import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerMetrics;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -226,11 +225,11 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
   }
 
   @Override
-  public synchronized void setClusterStatus(ClusterStatus st) {
-    super.setClusterStatus(st);
+  public synchronized void setClusterMetrics(ClusterMetrics st) {
+    super.setClusterMetrics(st);
     updateRegionLoad();
     for(CostFromRegionLoadFunction cost : regionLoadFunctions) {
-      cost.setClusterStatus(st);
+      cost.setClusterMetrics(st);
     }
 
     // update metrics size
@@ -527,23 +526,19 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     Map<String, Deque<BalancerRegionLoad>> oldLoads = loads;
     loads = new HashMap<>();
 
-    for (ServerName sn : clusterStatus.getServers()) {
-      ServerLoad sl = clusterStatus.getLoad(sn);
-      if (sl == null) {
-        continue;
-      }
-      for (Entry<byte[], RegionLoad> entry : sl.getRegionsLoad().entrySet()) {
-        Deque<BalancerRegionLoad> rLoads = oldLoads.get(Bytes.toString(entry.getKey()));
+    clusterStatus.getLiveServerMetrics().forEach((ServerName sn, ServerMetrics sm) -> {
+      sm.getRegionMetrics().forEach((byte[] regionName, RegionMetrics rm) -> {
+        Deque<BalancerRegionLoad> rLoads = oldLoads.get(Bytes.toString(regionName));
         if (rLoads == null) {
           // There was nothing there
           rLoads = new ArrayDeque<>();
         } else if (rLoads.size() >= numRegionLoadsToRemember) {
           rLoads.remove();
         }
-        rLoads.add(new BalancerRegionLoad(entry.getValue()));
-        loads.put(Bytes.toString(entry.getKey()), rLoads);
-      }
-    }
+        rLoads.add(new BalancerRegionLoad(rm));
+        loads.put(Bytes.toString(regionName), rLoads);
+      });
+    });
 
     for(CostFromRegionLoadFunction cost : regionLoadFunctions) {
       cost.setLoads(loads);
@@ -1371,14 +1366,14 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
    */
   abstract static class CostFromRegionLoadFunction extends CostFunction {
 
-    private ClusterStatus clusterStatus = null;
+    private ClusterMetrics clusterStatus = null;
     private Map<String, Deque<BalancerRegionLoad>> loads = null;
     private double[] stats = null;
     CostFromRegionLoadFunction(Configuration conf) {
       super(conf);
     }
 
-    void setClusterStatus(ClusterStatus status) {
+    void setClusterMetrics(ClusterMetrics status) {
       this.clusterStatus = status;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 4110dfd..602af91 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -22,7 +22,6 @@ import com.google.protobuf.Message;
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
-
 import java.io.IOException;
 import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
@@ -38,7 +37,6 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index a012a4e..31208c1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -51,8 +51,8 @@ import org.apache.commons.lang3.time.StopWatch;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.ChoreService;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -1168,8 +1168,7 @@ public final class Canary implements Tool {
     private void checkWriteTableDistribution() throws IOException {
       if (!admin.tableExists(writeTableName)) {
         int numberOfServers =
-            admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()
-                .size();
+            admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().size();
         if (numberOfServers == 0) {
           throw new IllegalStateException("No live regionservers");
         }
@@ -1180,10 +1179,10 @@ public final class Canary implements Tool {
         admin.enableTable(writeTableName);
       }
 
-      ClusterStatus status =
-          admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS, Option.MASTER));
-      int numberOfServers = status.getServersSize();
-      if (status.getServers().contains(status.getMaster())) {
+      ClusterMetrics status =
+          admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS, Option.MASTER));
+      int numberOfServers = status.getLiveServerMetrics().size();
+      if (status.getLiveServerMetrics().containsKey(status.getMasterName())) {
         numberOfServers -= 1;
       }
 
@@ -1502,8 +1501,8 @@ public final class Canary implements Tool {
         }
 
         // get any live regionservers not serving any regions
-        for (ServerName rs : this.admin
-            .getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
+        for (ServerName rs : this.admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+          .getLiveServerMetrics().keySet()) {
           String rsName = rs.getHostname();
           if (!rsAndRMap.containsKey(rsName)) {
             rsAndRMap.put(rsName, Collections.<RegionInfo> emptyList());

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 42faa9f..5ca1ed6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -72,8 +72,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
@@ -224,7 +224,7 @@ public class HBaseFsck extends Configured implements Closeable {
    * Internal resources
    **********************/
   private static final Logger LOG = LoggerFactory.getLogger(HBaseFsck.class.getName());
-  private ClusterStatus status;
+  private ClusterMetrics status;
   private ClusterConnection connection;
   private Admin admin;
   private Table meta;
@@ -525,7 +525,7 @@ public class HBaseFsck extends Configured implements Closeable {
     connection = (ClusterConnection)ConnectionFactory.createConnection(getConf());
     admin = connection.getAdmin();
     meta = connection.getTable(TableName.META_TABLE_NAME);
-    status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS,
+    status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS,
       Option.DEAD_SERVERS, Option.MASTER, Option.BACKUP_MASTERS,
       Option.REGIONS_IN_TRANSITION, Option.HBASE_VERSION));
   }
@@ -535,7 +535,7 @@ public class HBaseFsck extends Configured implements Closeable {
    */
   private void loadDeployedRegions() throws IOException, InterruptedException {
     // From the master, get a list of all known live region servers
-    Collection<ServerName> regionServers = status.getServers();
+    Collection<ServerName> regionServers = status.getLiveServerMetrics().keySet();
     errors.print("Number of live region servers: " + regionServers.size());
     if (details) {
       for (ServerName rsinfo: regionServers) {
@@ -553,10 +553,10 @@ public class HBaseFsck extends Configured implements Closeable {
     }
 
     // Print the current master name and state
-    errors.print("Master: " + status.getMaster());
+    errors.print("Master: " + status.getMasterName());
 
     // Print the list of all backup masters
-    Collection<ServerName> backupMasters = status.getBackupMasters();
+    Collection<ServerName> backupMasters = status.getBackupMasterNames();
     errors.print("Number of backup masters: " + backupMasters.size());
     if (details) {
       for (ServerName name: backupMasters) {
@@ -566,7 +566,7 @@ public class HBaseFsck extends Configured implements Closeable {
 
     errors.print("Average load: " + status.getAverageLoad());
     errors.print("Number of requests: " + status.getRequestCount());
-    errors.print("Number of regions: " + status.getRegionsCount());
+    errors.print("Number of regions: " + status.getRegionCount());
 
     List<RegionState> rits = status.getRegionStatesInTransition();
     errors.print("Number of regions in transition: " + rits.size());
@@ -2451,7 +2451,8 @@ public class HBaseFsck extends Configured implements Closeable {
         LOG.info("Patching hbase:meta with .regioninfo: " + hbi.getHdfsHRI());
         int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
         HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
-            admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers(), numReplicas);
+            admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+              .getLiveServerMetrics().keySet(), numReplicas);
 
         tryAssignmentRepair(hbi, "Trying to reassign region...");
       }
@@ -2478,7 +2479,8 @@ public class HBaseFsck extends Configured implements Closeable {
         LOG.info("Patching hbase:meta with with .regioninfo: " + hbi.getHdfsHRI());
         int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
         HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
-            admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers(), numReplicas);
+            admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+              .getLiveServerMetrics().keySet(), numReplicas);
         tryAssignmentRepair(hbi, "Trying to fix unassigned region...");
       }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
index 64f2766..86cb921 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
@@ -118,7 +118,7 @@ public class HBaseFsckRepair {
     while (EnvironmentEdgeManager.currentTime() < expiration) {
       try {
         boolean inTransition = false;
-        for (RegionState rs : admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
+        for (RegionState rs : admin.getClusterMetrics(EnumSet.of(Option.REGIONS_IN_TRANSITION))
                                    .getRegionStatesInTransition()) {
           if (RegionInfo.COMPARATOR.compare(rs.getRegion(), region) == 0) {
             inTransition = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
index 75c7dd5..16c256a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
@@ -747,7 +747,7 @@ public class RegionMover extends AbstractHBaseTool {
    * @throws IOException
    */
   private void stripMaster(ArrayList<String> regionServers, Admin admin) throws IOException {
-    ServerName master = admin.getClusterStatus(EnumSet.of(Option.MASTER)).getMaster();
+    ServerName master = admin.getClusterMetrics(EnumSet.of(Option.MASTER)).getMasterName();
     String masterHostname = master.getHostname();
     int masterPort = master.getPort();
     try {
@@ -825,7 +825,7 @@ public class RegionMover extends AbstractHBaseTool {
    */
   private ArrayList<String> getServers(Admin admin) throws IOException {
     ArrayList<ServerName> serverInfo = new ArrayList<>(
-        admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers());
+        admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet());
     ArrayList<String> regionServers = new ArrayList<>(serverInfo.size());
     for (ServerName server : serverInfo) {
       regionServers.add(server.getServerName().toLowerCase());

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
index 2902158..7b9cbb6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
@@ -41,8 +41,8 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -437,8 +437,8 @@ public class RegionSplitter {
    */
   private static int getRegionServerCount(final Connection connection) throws IOException {
     try (Admin admin = connection.getAdmin()) {
-      ClusterStatus status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
-      Collection<ServerName> servers = status.getServers();
+      ClusterMetrics status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS));
+      Collection<ServerName> servers = status.getLiveServerMetrics().keySet();
       return servers == null || servers.isEmpty()? 0: servers.size();
     }
   }


[3/3] hbase git commit: HBASE-19596 RegionMetrics/ServerMetrics/ClusterMetrics should apply to all public classes

Posted by ch...@apache.org.
HBASE-19596 RegionMetrics/ServerMetrics/ClusterMetrics should apply to all public classes


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

Branch: refs/heads/branch-2
Commit: 654edc5fa586a4b9c3e225a17836a6acc1d32928
Parents: 72631a0
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Thu Jan 4 12:40:09 2018 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Thu Jan 4 13:05:21 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/RegionMetricsBuilder.java      |   8 +
 .../org/apache/hadoop/hbase/client/Admin.java   | 119 ++---
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  44 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |  20 +-
 .../hbase/client/ClusterStatusListener.java     |  13 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  48 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  49 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  27 --
 .../hadoop/hbase/client/TestInterfaceAlign.java |   3 +-
 .../hadoop/hbase/DistributedHBaseCluster.java   |  56 +--
 .../hadoop/hbase/IntegrationTestIngest.java     |   6 +-
 .../hbase/IntegrationTestLazyCfLoading.java     |  10 +-
 ...IntegrationTestRegionReplicaReplication.java |   7 +-
 .../hadoop/hbase/IntegrationTestingUtility.java |   7 +-
 .../StripeCompactionsPerformanceEvaluation.java |  10 +-
 .../hadoop/hbase/chaos/actions/Action.java      |  38 +-
 .../chaos/actions/BatchRestartRsAction.java     |   5 +-
 .../chaos/actions/DumpClusterStatusAction.java  |   2 +-
 .../chaos/actions/MoveRegionsOfTableAction.java |   2 +-
 .../actions/RestartActiveMasterAction.java      |   2 +-
 .../actions/RestartRsHoldingMetaAction.java     |   6 +-
 .../UnbalanceKillAndRebalanceAction.java        |   7 +-
 .../chaos/actions/UnbalanceRegionsAction.java   |   7 +-
 .../mapreduce/IntegrationTestBulkLoad.java      |   5 +-
 .../hadoop/hbase/mttr/IntegrationTestMTTR.java  |   4 +-
 .../test/IntegrationTestBigLinkedList.java      |   7 +-
 ...stTimeBoundedRequestsWithRegionReplicas.java |   5 +-
 .../hbase/mapreduce/RegionSizeCalculator.java   |  15 +-
 .../mapreduce/TestRegionSizeCalculator.java     |  42 +-
 .../rest/StorageClusterStatusResource.java      |  48 +-
 .../rest/StorageClusterVersionResource.java     |   2 +-
 .../hbase/rsgroup/RSGroupBasedLoadBalancer.java |  11 +-
 .../hadoop/hbase/rsgroup/TestRSGroupsBase.java  |  57 +--
 .../tmpl/master/BackupMasterStatusTmpl.jamon    |   4 +-
 .../hbase/coprocessor/MasterObserver.java       |   9 +-
 .../hbase/master/ClusterStatusPublisher.java    |  33 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  36 +-
 .../hadoop/hbase/master/LoadBalancer.java       |   8 +-
 .../hbase/master/MasterCoprocessorHost.java     |  14 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   3 +-
 .../hbase/master/RegionPlacementMaintainer.java | 451 ++++++++++---------
 .../master/balancer/BalancerRegionLoad.java     |  17 +-
 .../hbase/master/balancer/BaseLoadBalancer.java |   8 +-
 .../master/balancer/ClusterStatusChore.java     |   2 +-
 .../master/balancer/RegionLocationFinder.java   |  10 +-
 .../master/balancer/StochasticLoadBalancer.java |  35 +-
 .../hbase/security/access/AccessController.java |   2 -
 .../org/apache/hadoop/hbase/tool/Canary.java    |  17 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |  20 +-
 .../hadoop/hbase/util/HBaseFsckRepair.java      |   2 +-
 .../apache/hadoop/hbase/util/RegionMover.java   |   4 +-
 .../hadoop/hbase/util/RegionSplitter.java       |   6 +-
 .../org/apache/hadoop/hbase/HBaseCluster.java   |  24 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |   5 +-
 .../apache/hadoop/hbase/MiniHBaseCluster.java   |  31 +-
 .../hadoop/hbase/TestClientClusterMetrics.java  |  29 +-
 .../hadoop/hbase/TestClientClusterStatus.java   |  51 +--
 .../org/apache/hadoop/hbase/TestRegionLoad.java |  21 +-
 .../apache/hadoop/hbase/TestRegionMetrics.java  |  14 +-
 .../hbase/TestStochasticBalancerJmxMetrics.java |  17 +-
 .../apache/hadoop/hbase/client/TestAdmin2.java  |   3 +-
 .../hbase/client/TestAsyncClusterAdminApi.java  |  45 +-
 .../client/TestAsyncDecommissionAdminApi.java   |   3 +-
 .../hadoop/hbase/client/TestFromClientSide.java |   7 +-
 .../hbase/client/TestMetaWithReplicas.java      |  10 +-
 .../hadoop/hbase/client/TestMultiParallel.java  |   2 +-
 .../hbase/coprocessor/TestMasterObserver.java   |   4 +-
 .../hadoop/hbase/master/TestMasterFailover.java |  29 +-
 .../TestMasterFailoverBalancerPersistence.java  |  25 +-
 .../TestMasterOperationsForRegionReplicas.java  |   6 +-
 .../hadoop/hbase/master/TestMasterShutdown.java |   8 +-
 .../TestFavoredStochasticBalancerPickers.java   |   8 +-
 .../TestFavoredStochasticLoadBalancer.java      |  27 +-
 .../balancer/TestRegionLocationFinder.java      |   3 +-
 .../balancer/TestStochasticLoadBalancer.java    |  32 +-
 .../TestRegionServerReadRequestMetrics.java     |  10 +-
 .../TestSplitTransactionOnCluster.java          |   6 +-
 .../replication/TestMasterReplication.java      |   9 +-
 .../replication/TestReplicationStatus.java      |   5 +-
 .../security/access/TestAccessController.java   |   7 +-
 .../hbase/snapshot/SnapshotTestingUtils.java    |   9 +-
 .../hadoop/hbase/util/BaseTestHBaseFsck.java    |   8 +-
 .../util/TestMiniClusterLoadSequential.java     |   4 +-
 83 files changed, 902 insertions(+), 933 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java
index b5d8048..f2a549a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java
@@ -29,12 +29,20 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 
 @InterfaceAudience.Private
 public final class RegionMetricsBuilder {
 
+  public static List<RegionMetrics> toRegionMetrics(
+      AdminProtos.GetRegionLoadResponse regionLoadResponse) {
+    return regionLoadResponse.getRegionLoadsList().stream()
+        .map(RegionMetricsBuilder::toRegionMetrics).collect(Collectors.toList());
+  }
+
   public static RegionMetrics toRegionMetrics(ClusterStatusProtos.RegionLoad regionLoadPB) {
     return RegionMetricsBuilder
         .newBuilder(regionLoadPB.getRegionSpecifier().getValue().toByteArray())

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index a94593c..f61b32e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -25,19 +25,19 @@ import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
 import java.util.concurrent.Future;
 import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CacheEvictionStats;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
-import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.RegionMetrics;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
@@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -1452,22 +1451,45 @@ public interface Admin extends Abortable, Closeable {
    * </pre>
    * @return cluster status
    * @throws IOException if a remote or network exception occurs
+   * @deprecated since 2.0 version and will be removed in 3.0 version.
+   *             use {@link #getClusterMetrics()}
+   */
+  @Deprecated
+  default ClusterStatus getClusterStatus() throws IOException {
+    return new ClusterStatus(getClusterMetrics());
+  }
+
+  /**
+   * Get whole cluster metrics, containing status about:
+   * <pre>
+   * hbase version
+   * cluster id
+   * primary/backup master(s)
+   * master's coprocessors
+   * live/dead regionservers
+   * balancer
+   * regions in transition
+   * </pre>
+   * @return cluster metrics
+   * @throws IOException if a remote or network exception occurs
    */
-  ClusterStatus getClusterStatus() throws IOException;
+  default ClusterMetrics getClusterMetrics() throws IOException {
+    return getClusterMetrics(EnumSet.allOf(ClusterMetrics.Option.class));
+  }
 
   /**
    * Get cluster status with a set of {@link Option} to get desired status.
    * @return cluster status
    * @throws IOException if a remote or network exception occurs
    */
-  ClusterStatus getClusterStatus(EnumSet<Option> options) throws IOException;
+  ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException;
 
   /**
    * @return current master server name
    * @throws IOException if a remote or network exception occurs
    */
   default ServerName getMaster() throws IOException {
-    return getClusterStatus(EnumSet.of(Option.MASTER)).getMaster();
+    return getClusterMetrics(EnumSet.of(Option.MASTER)).getMasterName();
   }
 
   /**
@@ -1475,7 +1497,7 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if a remote or network exception occurs
    */
   default Collection<ServerName> getBackupMasters() throws IOException {
-    return getClusterStatus(EnumSet.of(Option.BACKUP_MASTERS)).getBackupMasters();
+    return getClusterMetrics(EnumSet.of(Option.BACKUP_MASTERS)).getBackupMasterNames();
   }
 
   /**
@@ -1483,64 +1505,30 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if a remote or network exception occurs
    */
   default Collection<ServerName> getRegionServers() throws IOException {
-    return getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers();
+    return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet();
   }
 
   /**
-   * Get {@link RegionLoad} of all regions hosted on a regionserver.
+   * Get {@link RegionMetrics} of all regions hosted on a regionserver.
    *
-   * @param serverName region server from which regionload is required.
-   * @return region load map of all regions hosted on a region server
+   * @param serverName region server from which {@link RegionMetrics} is required.
+   * @return a {@link RegionMetrics} list of all regions hosted on a region server
    * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #getRegionLoads(ServerName)}
    */
-  @Deprecated
-  default Map<byte[], RegionLoad> getRegionLoad(ServerName serverName) throws IOException {
-    return getRegionLoad(serverName, null);
+  default List<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException {
+    return getRegionMetrics(serverName, null);
   }
 
   /**
-   * Get {@link RegionLoad} of all regions hosted on a regionserver.
+   * Get {@link RegionMetrics} of all regions hosted on a regionserver for a table.
    *
-   * @param serverName region server from which regionload is required.
-   * @return a region load list of all regions hosted on a region server
+   * @param serverName region server from which {@link RegionMetrics} is required.
+   * @param tableName get {@link RegionMetrics} of regions belonging to the table
+   * @return region metrics map of all regions of a table hosted on a region server
    * @throws IOException if a remote or network exception occurs
    */
-  default List<RegionLoad> getRegionLoads(ServerName serverName) throws IOException {
-    return getRegionLoads(serverName, null);
-  }
-
-  /**
-   * Get {@link RegionLoad} of all regions hosted on a regionserver for a table.
-   *
-   * @param serverName region server from which regionload is required.
-   * @param tableName get region load of regions belonging to the table
-   * @return region load map of all regions of a table hosted on a region server
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #getRegionLoads(ServerName, TableName)}
-   */
-  @Deprecated
-  default Map<byte[], RegionLoad> getRegionLoad(ServerName serverName, TableName tableName)
-      throws IOException {
-    List<RegionLoad> regionLoads = getRegionLoads(serverName, tableName);
-    Map<byte[], RegionLoad> resultMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for (RegionLoad regionLoad : regionLoads) {
-      resultMap.put(regionLoad.getName(), regionLoad);
-    }
-    return resultMap;
-  }
-
-  /**
-   * Get {@link RegionLoad} of all regions hosted on a regionserver for a table.
-   *
-   * @param serverName region server from which regionload is required.
-   * @param tableName get region load of regions belonging to the table
-   * @return region load map of all regions of a table hosted on a region server
-   * @throws IOException if a remote or network exception occurs
-   */
-  List<RegionLoad> getRegionLoads(ServerName serverName, TableName tableName) throws IOException;
+  List<RegionMetrics> getRegionMetrics(ServerName serverName,
+    TableName tableName) throws IOException;
 
   /**
    * @return Configuration used by the instance.
@@ -1772,11 +1760,26 @@ public interface Admin extends Abortable, Closeable {
   void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException;
 
   /**
-   * Helper that delegates to getClusterStatus().getMasterCoprocessors().
+   * Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
+   * @return an array of master coprocessors
+   * @see org.apache.hadoop.hbase.ClusterMetrics#getMasterCoprocessorNames()
+   * @deprecated since 2.0 version and will be removed in 3.0 version.
+   *             use {@link #getMasterCoprocessorNames()}
+   */
+  @Deprecated
+  default String[] getMasterCoprocessors() throws IOException {
+    return getMasterCoprocessorNames().stream().toArray(size -> new String[size]);
+  }
+
+  /**
+   * Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
    * @return an array of master coprocessors
-   * @see org.apache.hadoop.hbase.ClusterStatus#getMasterCoprocessors()
+   * @see org.apache.hadoop.hbase.ClusterMetrics#getMasterCoprocessorNames()
    */
-  String[] getMasterCoprocessors() throws IOException;
+  default List<String> getMasterCoprocessorNames() throws IOException {
+    return getClusterMetrics(EnumSet.of(Option.MASTER_COPROCESSORS))
+      .getMasterCoprocessorNames();
+  }
 
   /**
    * Get the current compaction state of a table. It could be in a major compaction, a minor
@@ -2371,7 +2374,7 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException
    */
   default int getMasterInfoPort() throws IOException {
-    return getClusterStatus(EnumSet.of(Option.MASTER_INFO_PORT)).getMasterInfoPort();
+    return getClusterMetrics(EnumSet.of(Option.MASTER_INFO_PORT)).getMasterInfoPort();
   }
 
   /**
@@ -2617,7 +2620,7 @@ public interface Admin extends Abortable, Closeable {
    * @return List of dead region servers.
    */
   default List<ServerName> listDeadServers() throws IOException {
-    return getClusterStatus(EnumSet.of(Option.DEAD_SERVERS)).getDeadServerNames();
+    return getClusterMetrics(EnumSet.of(Option.DEAD_SERVERS)).getDeadServerNames();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index e976033..af39f17 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.client;
 
 import com.google.protobuf.RpcChannel;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
@@ -28,10 +27,10 @@ import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Function;
 import java.util.regex.Pattern;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.RegionMetrics;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
@@ -886,40 +885,42 @@ public interface AsyncAdmin {
   /**
    * @return cluster status wrapped by {@link CompletableFuture}
    */
-  CompletableFuture<ClusterStatus> getClusterStatus();
+  CompletableFuture<ClusterMetrics> getClusterMetrics();
 
   /**
    * @return cluster status wrapped by {@link CompletableFuture}
    */
-  CompletableFuture<ClusterStatus> getClusterStatus(EnumSet<Option> options);
+  CompletableFuture<ClusterMetrics> getClusterMetrics(EnumSet<Option> options);
 
   /**
    * @return current master server name wrapped by {@link CompletableFuture}
    */
   default CompletableFuture<ServerName> getMaster() {
-    return getClusterStatus(EnumSet.of(Option.MASTER)).thenApply(ClusterStatus::getMaster);
+    return getClusterMetrics(EnumSet.of(Option.MASTER)).thenApply(ClusterMetrics::getMasterName);
   }
 
   /**
    * @return current backup master list wrapped by {@link CompletableFuture}
    */
   default CompletableFuture<Collection<ServerName>> getBackupMasters() {
-    return getClusterStatus(EnumSet.of(Option.BACKUP_MASTERS)).thenApply(ClusterStatus::getBackupMasters);
+    return getClusterMetrics(EnumSet.of(Option.BACKUP_MASTERS))
+      .thenApply(ClusterMetrics::getBackupMasterNames);
   }
 
   /**
    * @return current live region servers list wrapped by {@link CompletableFuture}
    */
   default CompletableFuture<Collection<ServerName>> getRegionServers() {
-    return getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).thenApply(ClusterStatus::getServers);
+    return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+      .thenApply(cm -> cm.getLiveServerMetrics().keySet());
   }
 
   /**
    * @return a list of master coprocessors wrapped by {@link CompletableFuture}
    */
-  default CompletableFuture<List<String>> getMasterCoprocessors() {
-    return getClusterStatus(EnumSet.of(Option.MASTER_COPROCESSORS))
-        .thenApply(ClusterStatus::getMasterCoprocessors).thenApply(Arrays::asList);
+  default CompletableFuture<List<String>> getMasterCoprocessorNames() {
+    return getClusterMetrics(EnumSet.of(Option.MASTER_COPROCESSORS))
+        .thenApply(ClusterMetrics::getMasterCoprocessorNames);
   }
 
   /**
@@ -927,8 +928,8 @@ public interface AsyncAdmin {
    * @return master info port
    */
   default CompletableFuture<Integer> getMasterInfoPort() {
-    return getClusterStatus(EnumSet.of(Option.MASTER_INFO_PORT)).thenApply(
-      ClusterStatus::getMasterInfoPort);
+    return getClusterMetrics(EnumSet.of(Option.MASTER_INFO_PORT)).thenApply(
+      ClusterMetrics::getMasterInfoPort);
   }
 
   /**
@@ -978,19 +979,20 @@ public interface AsyncAdmin {
   CompletableFuture<Void> clearCompactionQueues(ServerName serverName, Set<String> queues);
 
   /**
-   * Get a list of {@link RegionLoad} of all regions hosted on a region seerver.
+   * Get a list of {@link RegionMetrics} of all regions hosted on a region seerver.
    * @param serverName
-   * @return a list of {@link RegionLoad} wrapped by {@link CompletableFuture}
+   * @return a list of {@link RegionMetrics} wrapped by {@link CompletableFuture}
    */
-  CompletableFuture<List<RegionLoad>> getRegionLoads(ServerName serverName);
+  CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName);
 
   /**
-   * Get a list of {@link RegionLoad} of all regions hosted on a region seerver for a table.
+   * Get a list of {@link RegionMetrics} of all regions hosted on a region seerver for a table.
    * @param serverName
    * @param tableName
-   * @return a list of {@link RegionLoad} wrapped by {@link CompletableFuture}
+   * @return a list of {@link RegionMetrics} wrapped by {@link CompletableFuture}
    */
-  CompletableFuture<List<RegionLoad>> getRegionLoads(ServerName serverName, TableName tableName);
+  CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName,
+    TableName tableName);
 
   /**
    * Check whether master is in maintenance mode
@@ -1199,8 +1201,8 @@ public interface AsyncAdmin {
    * List all the dead region servers.
    */
   default CompletableFuture<List<ServerName>> listDeadServers() {
-    return this.getClusterStatus(EnumSet.of(Option.DEAD_SERVERS))
-        .thenApply(ClusterStatus::getDeadServerNames);
+    return this.getClusterMetrics(EnumSet.of(Option.DEAD_SERVERS))
+        .thenApply(ClusterMetrics::getDeadServerNames);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index e60e422..3ca5d69 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -27,10 +27,10 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.function.Function;
 import java.util.regex.Pattern;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.RegionMetrics;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
@@ -566,13 +566,13 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<ClusterStatus> getClusterStatus() {
-    return getClusterStatus(EnumSet.allOf(Option.class));
+  public CompletableFuture<ClusterMetrics> getClusterMetrics() {
+    return getClusterMetrics(EnumSet.allOf(Option.class));
   }
 
   @Override
-  public CompletableFuture<ClusterStatus> getClusterStatus(EnumSet<Option> options) {
-    return wrap(rawAdmin.getClusterStatus(options));
+  public CompletableFuture<ClusterMetrics> getClusterMetrics(EnumSet<Option> options) {
+    return wrap(rawAdmin.getClusterMetrics(options));
   }
 
   @Override
@@ -616,14 +616,14 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<List<RegionLoad>> getRegionLoads(ServerName serverName) {
-    return wrap(rawAdmin.getRegionLoads(serverName));
+  public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName) {
+    return wrap(rawAdmin.getRegionMetrics(serverName));
   }
 
   @Override
-  public CompletableFuture<List<RegionLoad>> getRegionLoads(ServerName serverName,
+  public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName,
       TableName tableName) {
-    return wrap(rawAdmin.getRegionLoads(serverName, tableName));
+    return wrap(rawAdmin.getRegionMetrics(serverName, tableName));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
index f2dae7a..7361238 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
@@ -19,8 +19,6 @@
 
 package org.apache.hadoop.hbase.client;
 
-
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
@@ -30,9 +28,9 @@ import java.net.NetworkInterface;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
+import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
@@ -42,6 +40,7 @@ import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.io.netty.bootstrap.Bootstrap;
 import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufInputStream;
 import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
@@ -52,7 +51,7 @@ import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
 import org.apache.hbase.thirdparty.io.netty.channel.socket.DatagramChannel;
 import org.apache.hbase.thirdparty.io.netty.channel.socket.DatagramPacket;
 import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioDatagramChannel;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 
 /**
@@ -133,7 +132,7 @@ class ClusterStatusListener implements Closeable {
    *
    * @param ncs the cluster status
    */
-  public void receive(ClusterStatus ncs) {
+  public void receive(ClusterMetrics ncs) {
     if (ncs.getDeadServerNames() != null) {
       for (ServerName sn : ncs.getDeadServerNames()) {
         if (!isDeadServer(sn)) {
@@ -264,7 +263,7 @@ class ClusterStatusListener implements Closeable {
         ByteBufInputStream bis = new ByteBufInputStream(dp.content());
         try {
           ClusterStatusProtos.ClusterStatus csp = ClusterStatusProtos.ClusterStatus.parseFrom(bis);
-          ClusterStatus ncs = ProtobufUtil.toClusterStatus(csp);
+          ClusterMetrics ncs = ClusterMetricsBuilder.toClusterMetrics(csp);
           receive(ncs);
         } finally {
           bis.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 9937adf..4ac1c21 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -46,8 +46,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
@@ -59,8 +60,9 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.RegionMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
@@ -2072,29 +2074,31 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public ClusterStatus getClusterStatus() throws IOException {
-    return getClusterStatus(EnumSet.allOf(Option.class));
-  }
-
-  @Override
-  public ClusterStatus getClusterStatus(EnumSet<Option> options) throws IOException {
-    return executeCallable(new MasterCallable<ClusterStatus>(getConnection(),
+  public ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException {
+    return executeCallable(new MasterCallable<ClusterMetrics>(getConnection(),
         this.rpcControllerFactory) {
       @Override
-      protected ClusterStatus rpcCall() throws Exception {
+      protected ClusterMetrics rpcCall() throws Exception {
         GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest(options);
-        return ProtobufUtil.toClusterStatus(
+        return ClusterMetricsBuilder.toClusterMetrics(
           master.getClusterStatus(getRpcController(), req).getClusterStatus());
       }
     });
   }
 
   @Override
-  public List<RegionLoad> getRegionLoads(ServerName serverName, TableName tableName)
+  public List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName)
       throws IOException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
     HBaseRpcController controller = rpcControllerFactory.newController();
-    return ProtobufUtil.getRegionLoad(controller, admin, tableName);
+    AdminProtos.GetRegionLoadRequest request =
+      RequestConverter.buildGetRegionLoadRequest(tableName);
+    try {
+      return admin.getRegionLoad(controller, request).getRegionLoadsList().stream()
+        .map(RegionMetricsBuilder::toRegionMetrics).collect(Collectors.toList());
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
+    }
   }
 
   @Override
@@ -2429,16 +2433,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public String[] getMasterCoprocessors() {
-    try {
-      return getClusterStatus(EnumSet.of(Option.MASTER_COPROCESSORS)).getMasterCoprocessors();
-    } catch (IOException e) {
-      LOG.error("Could not getClusterStatus()",e);
-      return null;
-    }
-  }
-
-  @Override
   public CompactionState getCompactionState(final TableName tableName)
   throws IOException {
     return getCompactionState(tableName, CompactType.NORMAL);
@@ -3148,15 +3142,15 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public void updateConfiguration() throws IOException {
-    ClusterStatus status = getClusterStatus(
+    ClusterMetrics status = getClusterMetrics(
       EnumSet.of(Option.LIVE_SERVERS, Option.MASTER, Option.BACKUP_MASTERS));
-    for (ServerName server : status.getServers()) {
+    for (ServerName server : status.getLiveServerMetrics().keySet()) {
       updateConfiguration(server);
     }
 
-    updateConfiguration(status.getMaster());
+    updateConfiguration(status.getMasterName());
 
-    for (ServerName server : status.getBackupMasters()) {
+    for (ServerName server : status.getBackupMasterNames()) {
       updateConfiguration(server);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index ceda280..ac00234 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -42,15 +42,17 @@ import java.util.stream.Stream;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.RegionMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
@@ -2601,20 +2603,20 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<ClusterStatus> getClusterStatus() {
-    return getClusterStatus(EnumSet.allOf(Option.class));
+  public CompletableFuture<ClusterMetrics> getClusterMetrics() {
+    return getClusterMetrics(EnumSet.allOf(Option.class));
   }
 
   @Override
-  public CompletableFuture<ClusterStatus>getClusterStatus(EnumSet<Option> options) {
+  public CompletableFuture<ClusterMetrics> getClusterMetrics(EnumSet<Option> options) {
     return this
-        .<ClusterStatus> newMasterCaller()
+        .<ClusterMetrics> newMasterCaller()
         .action(
           (controller, stub) -> this
-              .<GetClusterStatusRequest, GetClusterStatusResponse, ClusterStatus> call(controller,
+              .<GetClusterStatusRequest, GetClusterStatusResponse, ClusterMetrics> call(controller,
                 stub, RequestConverter.buildGetClusterStatusRequest(options),
                 (s, c, req, done) -> s.getClusterStatus(c, req, done),
-                resp -> ProtobufUtil.toClusterStatus(resp.getClusterStatus()))).call();
+                resp -> ClusterMetricsBuilder.toClusterMetrics(resp.getClusterStatus()))).call();
   }
 
   @Override
@@ -2665,17 +2667,16 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Void> updateConfiguration() {
     CompletableFuture<Void> future = new CompletableFuture<Void>();
-    getClusterStatus(
-      EnumSet.of(Option.LIVE_SERVERS, Option.MASTER, Option.BACKUP_MASTERS))
-          .whenComplete(
-      (status, err) -> {
+    getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS, Option.MASTER, Option.BACKUP_MASTERS))
+      .whenComplete((status, err) -> {
         if (err != null) {
           future.completeExceptionally(err);
         } else {
           List<CompletableFuture<Void>> futures = new ArrayList<>();
-          status.getServers().forEach((server) -> futures.add(updateConfiguration(server)));
-          futures.add(updateConfiguration(status.getMaster()));
-          status.getBackupMasters().forEach(master -> futures.add(updateConfiguration(master)));
+          status.getLiveServerMetrics().keySet()
+              .forEach(server -> futures.add(updateConfiguration(server)));
+          futures.add(updateConfiguration(status.getMasterName()));
+          status.getBackupMasterNames().forEach(master -> futures.add(updateConfiguration(master)));
           CompletableFuture.allOf(futures.toArray(new CompletableFuture<?>[futures.size()]))
               .whenComplete((result, err2) -> {
                 if (err2 != null) {
@@ -2725,25 +2726,25 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<List<RegionLoad>> getRegionLoads(ServerName serverName) {
-    return getRegionLoad(GetRegionLoadRequest.newBuilder().build(), serverName);
+  public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName) {
+    return getRegionMetrics(GetRegionLoadRequest.newBuilder().build(), serverName);
   }
 
   @Override
-  public CompletableFuture<List<RegionLoad>> getRegionLoads(ServerName serverName,
+  public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName,
       TableName tableName) {
     Preconditions.checkNotNull(tableName,
       "tableName is null. If you don't specify a tableName, use getRegionLoads() instead");
-    return getRegionLoad(RequestConverter.buildGetRegionLoadRequest(tableName), serverName);
+    return getRegionMetrics(RequestConverter.buildGetRegionLoadRequest(tableName), serverName);
   }
 
-  private CompletableFuture<List<RegionLoad>> getRegionLoad(GetRegionLoadRequest request,
+  private CompletableFuture<List<RegionMetrics>> getRegionMetrics(GetRegionLoadRequest request,
       ServerName serverName) {
-    return this.<List<RegionLoad>> newAdminCaller()
+    return this.<List<RegionMetrics>> newAdminCaller()
         .action((controller, stub) -> this
-            .<GetRegionLoadRequest, GetRegionLoadResponse, List<RegionLoad>> adminCall(controller,
-              stub, request, (s, c, req, done) -> s.getRegionLoad(controller, req, done),
-              ProtobufUtil::getRegionLoadInfo))
+            .<GetRegionLoadRequest, GetRegionLoadResponse, List<RegionMetrics>>
+              adminCall(controller, stub, request, (s, c, req, done) ->
+                s.getRegionLoad(controller, req, done), RegionMetricsBuilder::toRegionMetrics))
         .serverName(serverName).call();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index cc30e53..f0a5d16 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -49,8 +49,6 @@ import org.apache.hadoop.hbase.Cell.Type;
 import org.apache.hadoop.hbase.CellBuilderType;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.ClusterMetricsBuilder;
-import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ExtendedCellBuilder;
 import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
@@ -126,7 +124,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRe
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
@@ -1764,20 +1761,6 @@ public final class ProtobufUtil {
     }
   }
 
-  public static List<org.apache.hadoop.hbase.RegionLoad> getRegionLoad(
-      final RpcController controller, final AdminService.BlockingInterface admin,
-      final TableName tableName) throws IOException {
-    GetRegionLoadRequest request =
-        RequestConverter.buildGetRegionLoadRequest(tableName);
-    GetRegionLoadResponse response;
-    try {
-      response = admin.getRegionLoad(controller, request);
-    } catch (ServiceException se) {
-      throw getRemoteException(se);
-    }
-    return getRegionLoadInfo(response);
-  }
-
   public static List<org.apache.hadoop.hbase.RegionLoad> getRegionLoadInfo(
       GetRegionLoadResponse regionLoadResponse) {
     List<org.apache.hadoop.hbase.RegionLoad> regionLoadList =
@@ -2969,16 +2952,6 @@ public final class ProtobufUtil {
         snapshotDesc.getCreationTime(), snapshotDesc.getVersion());
   }
 
-  /**
-   * Convert a protobuf ClusterStatus to a ClusterStatus
-   *
-   * @param proto the protobuf ClusterStatus
-   * @return the converted ClusterStatus
-   */
-  public static ClusterStatus toClusterStatus(ClusterStatusProtos.ClusterStatus proto) {
-    return new ClusterStatus(ClusterMetricsBuilder.toClusterMetrics(proto));
-  }
-
   public static RegionLoadStats createRegionLoadStats(ClientProtos.RegionLoadStats stats) {
     return new RegionLoadStats(stats.getMemStoreLoad(), stats.getHeapOccupancy(),
         stats.getCompactionPressure());

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
index 6318bc4..2266d06 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
@@ -75,7 +75,8 @@ public class TestInterfaceAlign {
   }
 
   private <T> List<String> getMethodNames(Class<T> c) {
-    return Arrays.asList(c.getDeclaredMethods()).stream().filter(m -> !isDeprecated(m))
+    // DON'T use the getDeclaredMethods as we want to check the Public APIs only.
+    return Arrays.asList(c.getMethods()).stream().filter(m -> !isDeprecated(m))
         .map(Method::getName).distinct().collect(Collectors.toList());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
index 6546ac9..943f2a6 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
@@ -25,22 +25,22 @@ import java.util.List;
 import java.util.Objects;
 import java.util.Set;
 import java.util.TreeSet;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterManager.ServiceType;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Threads;
 
 /**
  * Manages the interactions with an already deployed distributed cluster (as opposed to
@@ -65,7 +65,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
     this.clusterManager = clusterManager;
     this.connection = ConnectionFactory.createConnection(conf);
     this.admin = this.connection.getAdmin();
-    this.initialClusterStatus = getClusterStatus();
+    this.initialClusterStatus = getClusterMetrics();
   }
 
   public void setClusterManager(ClusterManager clusterManager) {
@@ -81,12 +81,12 @@ public class DistributedHBaseCluster extends HBaseCluster {
    * @throws IOException
    */
   @Override
-  public ClusterStatus getClusterStatus() throws IOException {
-    return admin.getClusterStatus();
+  public ClusterMetrics getClusterMetrics() throws IOException {
+    return admin.getClusterMetrics();
   }
 
   @Override
-  public ClusterStatus getInitialClusterStatus() throws IOException {
+  public ClusterMetrics getInitialClusterMetrics() throws IOException {
     return initialClusterStatus;
   }
 
@@ -315,8 +315,8 @@ public class DistributedHBaseCluster extends HBaseCluster {
   }
 
   @Override
-  public boolean restoreClusterStatus(ClusterStatus initial) throws IOException {
-    ClusterStatus current = getClusterStatus();
+  public boolean restoreClusterMetrics(ClusterMetrics initial) throws IOException {
+    ClusterMetrics current = getClusterMetrics();
 
     LOG.info("Restoring cluster - started");
 
@@ -330,15 +330,15 @@ public class DistributedHBaseCluster extends HBaseCluster {
     return success;
   }
 
-  protected boolean restoreMasters(ClusterStatus initial, ClusterStatus current) {
+  protected boolean restoreMasters(ClusterMetrics initial, ClusterMetrics current) {
     List<IOException> deferred = new ArrayList<>();
     //check whether current master has changed
-    final ServerName initMaster = initial.getMaster();
-    if (!ServerName.isSameAddress(initMaster, current.getMaster())) {
+    final ServerName initMaster = initial.getMasterName();
+    if (!ServerName.isSameAddress(initMaster, current.getMasterName())) {
       LOG.info("Restoring cluster - Initial active master : "
               + initMaster.getHostAndPort()
               + " has changed to : "
-              + current.getMaster().getHostAndPort());
+              + current.getMasterName().getHostAndPort());
       // If initial master is stopped, start it, before restoring the state.
       // It will come up as a backup master, if there is already an active master.
       try {
@@ -353,14 +353,14 @@ public class DistributedHBaseCluster extends HBaseCluster {
         // 1. Kill the current backups
         // 2. Stop current master
         // 3. Start backup masters
-        for (ServerName currentBackup : current.getBackupMasters()) {
+        for (ServerName currentBackup : current.getBackupMasterNames()) {
           if (!ServerName.isSameAddress(currentBackup, initMaster)) {
             LOG.info("Restoring cluster - stopping backup master: " + currentBackup);
             stopMaster(currentBackup);
           }
         }
-        LOG.info("Restoring cluster - stopping active master: " + current.getMaster());
-        stopMaster(current.getMaster());
+        LOG.info("Restoring cluster - stopping active master: " + current.getMasterName());
+        stopMaster(current.getMasterName());
         waitForActiveAndReadyMaster(); // wait so that active master takes over
       } catch (IOException ex) {
         // if we fail to start the initial active master, we do not want to continue stopping
@@ -369,7 +369,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
       }
 
       //start backup masters
-      for (ServerName backup : initial.getBackupMasters()) {
+      for (ServerName backup : initial.getBackupMasterNames()) {
         try {
           //these are not started in backup mode, but we should already have an active master
           if (!clusterManager.isRunning(ServiceType.HBASE_MASTER,
@@ -387,13 +387,13 @@ public class DistributedHBaseCluster extends HBaseCluster {
       //current master has not changed, match up backup masters
       Set<ServerName> toStart = new TreeSet<>(new ServerNameIgnoreStartCodeComparator());
       Set<ServerName> toKill = new TreeSet<>(new ServerNameIgnoreStartCodeComparator());
-      toStart.addAll(initial.getBackupMasters());
-      toKill.addAll(current.getBackupMasters());
+      toStart.addAll(initial.getBackupMasterNames());
+      toKill.addAll(current.getBackupMasterNames());
 
-      for (ServerName server : current.getBackupMasters()) {
+      for (ServerName server : current.getBackupMasterNames()) {
         toStart.remove(server);
       }
-      for (ServerName server: initial.getBackupMasters()) {
+      for (ServerName server: initial.getBackupMasterNames()) {
         toKill.remove(server);
       }
 
@@ -442,18 +442,18 @@ public class DistributedHBaseCluster extends HBaseCluster {
     }
   }
 
-  protected boolean restoreRegionServers(ClusterStatus initial, ClusterStatus current) {
+  protected boolean restoreRegionServers(ClusterMetrics initial, ClusterMetrics current) {
     Set<ServerName> toStart = new TreeSet<>(new ServerNameIgnoreStartCodeComparator());
     Set<ServerName> toKill = new TreeSet<>(new ServerNameIgnoreStartCodeComparator());
-    toStart.addAll(initial.getServers());
-    toKill.addAll(current.getServers());
+    toStart.addAll(initial.getLiveServerMetrics().keySet());
+    toKill.addAll(current.getLiveServerMetrics().keySet());
 
-    ServerName master = initial.getMaster();
+    ServerName master = initial.getMasterName();
 
-    for (ServerName server : current.getServers()) {
+    for (ServerName server : current.getLiveServerMetrics().keySet()) {
       toStart.remove(server);
     }
-    for (ServerName server: initial.getServers()) {
+    for (ServerName server: initial.getLiveServerMetrics().keySet()) {
       toKill.remove(server);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngest.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngest.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngest.java
index b4b1314..1850e91 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngest.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngest.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -160,7 +159,8 @@ public class IntegrationTestIngest extends IntegrationTestBase {
       int recordSize, int writeThreads, int readThreads) throws Exception {
 
     LOG.info("Running ingest");
-    LOG.info("Cluster size:" + util.getHBaseClusterInterface().getClusterStatus().getServersSize());
+    LOG.info("Cluster size:" + util.getHBaseClusterInterface()
+      .getClusterMetrics().getLiveServerMetrics().size());
 
     long start = System.currentTimeMillis();
     String runtimeKey = String.format(RUN_TIME_KEY, this.getClass().getSimpleName());
@@ -248,7 +248,7 @@ public class IntegrationTestIngest extends IntegrationTestBase {
   /** Estimates a data size based on the cluster size */
   protected long getNumKeys(long keysPerServer)
       throws IOException {
-    int numRegionServers = cluster.getClusterStatus().getServersSize();
+    int numRegionServers = cluster.getClusterMetrics().getLiveServerMetrics().size();
     return keysPerServer * numRegionServers;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
index 60dff7a..c2966a3 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
@@ -23,7 +23,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -31,17 +30,16 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.LoadTestKVGenerator;
 import org.apache.hadoop.hbase.util.MultiThreadedWriter;
 import org.apache.hadoop.hbase.util.RegionSplitter;
 import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
-import org.apache.hadoop.hbase.util.LoadTestKVGenerator;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -195,7 +193,8 @@ public class IntegrationTestLazyCfLoading {
       hcd.setDataBlockEncoding(blockEncoding);
       htd.addFamily(hcd);
     }
-    int serverCount = util.getHBaseClusterInterface().getClusterStatus().getServersSize();
+    int serverCount = util.getHBaseClusterInterface().getClusterMetrics()
+      .getLiveServerMetrics().size();
     byte[][] splits = new RegionSplitter.HexStringSplit().split(serverCount * REGIONS_PER_SERVER);
     util.getAdmin().createTable(htd, splits);
     LOG.info("Created table");
@@ -222,7 +221,8 @@ public class IntegrationTestLazyCfLoading {
     Configuration conf = util.getConfiguration();
     String timeoutKey = String.format(TIMEOUT_KEY, this.getClass().getSimpleName());
     long maxRuntime = conf.getLong(timeoutKey, DEFAULT_TIMEOUT_MINUTES);
-    long serverCount = util.getHBaseClusterInterface().getClusterStatus().getServersSize();
+    long serverCount = util.getHBaseClusterInterface().getClusterMetrics()
+      .getLiveServerMetrics().size();
     long keysToWrite = serverCount * KEYS_TO_WRITE_PER_SERVER;
     Connection connection = ConnectionFactory.createConnection(conf);
     Table table = connection.getTable(TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaReplication.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaReplication.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaReplication.java
index 4b37bbb..be2616a 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaReplication.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaReplication.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.ConstantDelayQueue;
@@ -54,8 +53,9 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
  * The job will run for <b>at least<b> given runtime (default 10min) by running a concurrent
  * writer and reader workload followed by a concurrent updater and reader workload for
  * num_keys_per_server.
- *<p>
+ * <p>
  * Example usage:
+ * </p>
  * <pre>
  * hbase org.apache.hadoop.hbase.IntegrationTestRegionReplicaReplication
  * -DIntegrationTestRegionReplicaReplication.num_keys_per_server=10000
@@ -154,7 +154,8 @@ public class IntegrationTestRegionReplicaReplication extends IntegrationTestInge
       int recordSize, int writeThreads, int readThreads) throws Exception {
 
     LOG.info("Running ingest");
-    LOG.info("Cluster size:" + util.getHBaseClusterInterface().getClusterStatus().getServersSize());
+    LOG.info("Cluster size:" + util.getHBaseClusterInterface()
+      .getClusterMetrics().getLiveServerMetrics().size());
 
     // sleep for some time so that the cache for disabled tables does not interfere.
     Threads.sleep(

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestingUtility.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestingUtility.java
index 8038758..afb21e4 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestingUtility.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestingUtility.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ReflectionUtils;
 
@@ -61,9 +60,9 @@ public class IntegrationTestingUtility extends HBaseTestingUtility {
 
   /** Config for pluggable hbase cluster manager */
   private static final String HBASE_CLUSTER_MANAGER_CLASS = "hbase.it.clustermanager.class";
-  private static final Class<? extends ClusterManager> DEFAULT_HBASE_CLUSTER_MANAGER_CLASS = 
+  private static final Class<? extends ClusterManager> DEFAULT_HBASE_CLUSTER_MANAGER_CLASS =
     HBaseClusterManager.class;
-  
+
   /**
    * Initializes the state of the cluster. It starts a new in-process mini cluster, OR
    * if we are given an already deployed distributed cluster it initializes the state.
@@ -85,7 +84,7 @@ public class IntegrationTestingUtility extends HBaseTestingUtility {
    */
   public void checkNodeCount(int numSlaves) throws Exception {
     HBaseCluster cluster = getHBaseClusterInterface();
-    if (cluster.getClusterStatus().getServers().size() < numSlaves) {
+    if (cluster.getClusterMetrics().getLiveServerMetrics().size() < numSlaves) {
       throw new Exception("Cluster does not have enough nodes:" + numSlaves);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
index 1dec028..216c992 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
@@ -21,10 +21,8 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 import java.util.Locale;
 import java.util.Set;
-
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -33,12 +31,13 @@ import org.apache.hadoop.hbase.regionserver.StripeStoreConfig;
 import org.apache.hadoop.hbase.regionserver.StripeStoreEngine;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.LoadTestKVGenerator;
 import org.apache.hadoop.hbase.util.MultiThreadedAction;
 import org.apache.hadoop.hbase.util.MultiThreadedReader;
 import org.apache.hadoop.hbase.util.MultiThreadedWriter;
 import org.apache.hadoop.hbase.util.RegionSplitter;
 import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
-import org.apache.hadoop.hbase.util.LoadTestKVGenerator;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -196,7 +195,8 @@ public class StripeCompactionsPerformanceEvaluation extends AbstractHBaseTool {
   }
 
   private void runOneTest(String description, Configuration conf) throws Exception {
-    int numServers = util.getHBaseClusterInterface().getClusterStatus().getServersSize();
+    int numServers = util.getHBaseClusterInterface()
+      .getClusterMetrics().getLiveServerMetrics().size();
     long startKey = preloadKeys * numServers;
     long endKey = startKey + writeKeys * numServers;
     status(String.format("%s test starting on %d servers; preloading 0 to %d and writing to %d",
@@ -298,7 +298,7 @@ public class StripeCompactionsPerformanceEvaluation extends AbstractHBaseTool {
       htd.setConfiguration(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, "1048576");
     }
     byte[][] splits = new RegionSplitter.HexStringSplit().split(
-        util.getHBaseClusterInterface().getClusterStatus().getServersSize());
+        util.getHBaseClusterInterface().getClusterMetrics().getLiveServerMetrics().size());
     util.getAdmin().createTable(htd, splits);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
index 7a89569..ae8cd1f 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
@@ -23,17 +23,17 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.function.BiConsumer;
 import java.util.function.Consumer;
-
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseCluster;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
-import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerMetrics;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
@@ -79,7 +79,7 @@ public class Action {
 
   protected ActionContext context;
   protected HBaseCluster cluster;
-  protected ClusterStatus initialStatus;
+  protected ClusterMetrics initialStatus;
   protected ServerName[] initialServers;
 
   protected long killMasterTimeout;
@@ -94,8 +94,8 @@ public class Action {
   public void init(ActionContext context) throws IOException {
     this.context = context;
     cluster = context.getHBaseCluster();
-    initialStatus = cluster.getInitialClusterStatus();
-    Collection<ServerName> regionServers = initialStatus.getServers();
+    initialStatus = cluster.getInitialClusterMetrics();
+    Collection<ServerName> regionServers = initialStatus.getLiveServerMetrics().keySet();
     initialServers = regionServers.toArray(new ServerName[regionServers.size()]);
 
     killMasterTimeout = cluster.getConf().getLong(KILL_MASTER_TIMEOUT_KEY,
@@ -118,13 +118,13 @@ public class Action {
 
   /** Returns current region servers - active master */
   protected ServerName[] getCurrentServers() throws IOException {
-    ClusterStatus clusterStatus = cluster.getClusterStatus();
-    Collection<ServerName> regionServers = clusterStatus.getServers();
+    ClusterMetrics clusterStatus = cluster.getClusterMetrics();
+    Collection<ServerName> regionServers = clusterStatus.getLiveServerMetrics().keySet();
     int count = regionServers == null ? 0 : regionServers.size();
     if (count <= 0) {
       return new ServerName [] {};
     }
-    ServerName master = clusterStatus.getMaster();
+    ServerName master = clusterStatus.getMasterName();
     if (master == null || !regionServers.contains(master)) {
       return regionServers.toArray(new ServerName[count]);
     }
@@ -156,7 +156,7 @@ public class Action {
     cluster.killRegionServer(server);
     cluster.waitForRegionServerToStop(server, killRsTimeout);
     LOG.info("Killed region server:" + server + ". Reported num of rs:"
-        + cluster.getClusterStatus().getServersSize());
+        + cluster.getClusterMetrics().getLiveServerMetrics().size());
   }
 
   protected void startRs(ServerName server) throws IOException {
@@ -164,7 +164,7 @@ public class Action {
     cluster.startRegionServer(server.getHostname(), server.getPort());
     cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), startRsTimeout);
     LOG.info("Started region server:" + server + ". Reported num of rs:"
-      + cluster.getClusterStatus().getServersSize());
+      + cluster.getClusterMetrics().getLiveServerMetrics().size());
   }
 
   protected void killZKNode(ServerName server) throws IOException {
@@ -172,7 +172,7 @@ public class Action {
     cluster.killZkNode(server);
     cluster.waitForZkNodeToStop(server, killZkNodeTimeout);
     LOG.info("Killed zookeeper node:" + server + ". Reported num of rs:"
-      + cluster.getClusterStatus().getServersSize());
+      + cluster.getClusterMetrics().getLiveServerMetrics().size());
   }
 
   protected void startZKNode(ServerName server) throws IOException {
@@ -187,7 +187,7 @@ public class Action {
     cluster.killDataNode(server);
     cluster.waitForDataNodeToStop(server, killDataNodeTimeout);
     LOG.info("Killed datanode:" + server + ". Reported num of rs:"
-      + cluster.getClusterStatus().getServersSize());
+      + cluster.getClusterMetrics().getLiveServerMetrics().size());
   }
 
   protected void startDataNode(ServerName server) throws IOException {
@@ -197,16 +197,18 @@ public class Action {
     LOG.info("Started datanode:" + server);
   }
 
-  protected void unbalanceRegions(ClusterStatus clusterStatus,
+  protected void unbalanceRegions(ClusterMetrics clusterStatus,
       List<ServerName> fromServers, List<ServerName> toServers,
       double fractionOfRegions) throws Exception {
     List<byte[]> victimRegions = new LinkedList<>();
-    for (ServerName server : fromServers) {
-      ServerLoad serverLoad = clusterStatus.getLoad(server);
+    for (Map.Entry<ServerName, ServerMetrics> entry
+      : clusterStatus.getLiveServerMetrics().entrySet()) {
+      ServerName sn = entry.getKey();
+      ServerMetrics serverLoad = entry.getValue();
       // Ugh.
-      List<byte[]> regions = new LinkedList<>(serverLoad.getRegionsLoad().keySet());
+      List<byte[]> regions = new LinkedList<>(serverLoad.getRegionMetrics().keySet());
       int victimRegionCount = (int)Math.ceil(fractionOfRegions * regions.size());
-      LOG.debug("Removing " + victimRegionCount + " regions from " + server.getServerName());
+      LOG.debug("Removing " + victimRegionCount + " regions from " + sn);
       for (int i = 0; i < victimRegionCount; ++i) {
         int victimIx = RandomUtils.nextInt(0, regions.size());
         String regionId = HRegionInfo.encodeRegionName(regions.remove(victimIx));

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.java
index 75414ae..31ec874 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.chaos.actions;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
 
@@ -61,7 +60,7 @@ public class BatchRestartRsAction extends RestartActionBaseAction {
     }
 
     LOG.info("Killed " + killedServers.size() + " region servers. Reported num of rs:"
-        + cluster.getClusterStatus().getServersSize());
+        + cluster.getClusterMetrics().getLiveServerMetrics().size());
 
     sleep(sleepTime);
 
@@ -76,6 +75,6 @@ public class BatchRestartRsAction extends RestartActionBaseAction {
           PolicyBasedChaosMonkey.TIMEOUT);
     }
     LOG.info("Started " + killedServers.size() +" region servers. Reported num of rs:"
-        + cluster.getClusterStatus().getServersSize());
+        + cluster.getClusterMetrics().getLiveServerMetrics().size());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.java
index 0403fe0..36b8530 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.java
@@ -33,6 +33,6 @@ public class DumpClusterStatusAction extends Action {
   @Override
   public void perform() throws Exception {
     LOG.debug("Performing action: Dump cluster status");
-    LOG.info("Cluster status\n" + cluster.getClusterStatus());
+    LOG.info("Cluster status\n" + cluster.getClusterMetrics());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
index 299e51b..08958e8 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
@@ -57,7 +57,7 @@ public class MoveRegionsOfTableAction extends Action {
 
     Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin();
     Collection<ServerName> serversList =
-        admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers();
+        admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet();
     ServerName[] servers = serversList.toArray(new ServerName[serversList.size()]);
 
     LOG.info("Performing action: Move regions of table " + tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.java
index a9bc23a..89415b9 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.java
@@ -31,7 +31,7 @@ public class RestartActiveMasterAction extends RestartActionBaseAction {
   public void perform() throws Exception {
     LOG.info("Performing action: Restart active master");
 
-    ServerName master = cluster.getClusterStatus().getMaster();
+    ServerName master = cluster.getClusterMetrics().getMasterName();
     restartMaster(master, sleepTime);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.java
index a6b4fc7..f17b806 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.java
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.hbase.chaos.actions;
 
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ServerName;
 
 /**
@@ -36,8 +36,8 @@ public class RestartRsHoldingMetaAction extends RestartActionBaseAction {
       LOG.warn("No server is holding hbase:meta right now.");
       return;
     }
-    ClusterStatus clusterStatus = cluster.getClusterStatus();
-    if (server.equals(clusterStatus.getMaster())) {
+    ClusterMetrics clusterStatus = cluster.getClusterMetrics();
+    if (server.equals(clusterStatus.getMasterName())) {
       // Master holds the meta, so restart the master.
       restartMaster(server, sleepTime);
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java
index 7b6f292..cab5aa2 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java
@@ -23,9 +23,8 @@ import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
-
 import org.apache.commons.lang3.RandomUtils;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ServerName;
 import org.junit.Assert;
 
@@ -54,8 +53,8 @@ public class UnbalanceKillAndRebalanceAction extends Action {
 
   @Override
   public void perform() throws Exception {
-    ClusterStatus status = this.cluster.getClusterStatus();
-    List<ServerName> victimServers = new LinkedList<>(status.getServers());
+    ClusterMetrics status = this.cluster.getClusterMetrics();
+    List<ServerName> victimServers = new LinkedList<>(status.getLiveServerMetrics().keySet());
     Set<ServerName> killedServers = new HashSet<>();
 
     int liveCount = (int)Math.ceil(FRC_SERVERS_THAT_HOARD_AND_LIVE * victimServers.size());

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java
index e5faf08..f85ff04 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java
@@ -21,9 +21,8 @@ package org.apache.hadoop.hbase.chaos.actions;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
-
 import org.apache.commons.lang3.RandomUtils;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ServerName;
 
 /**
@@ -47,8 +46,8 @@ public class UnbalanceRegionsAction extends Action {
   @Override
   public void perform() throws Exception {
     LOG.info("Unbalancing regions");
-    ClusterStatus status = this.cluster.getClusterStatus();
-    List<ServerName> victimServers = new LinkedList<>(status.getServers());
+    ClusterMetrics status = this.cluster.getClusterMetrics();
+    List<ServerName> victimServers = new LinkedList<>(status.getLiveServerMetrics().keySet());
     int targetServerCount = (int)Math.ceil(fractionOfServers * victimServers.size());
     List<ServerName> targetServers = new ArrayList<>(targetServerCount);
     for (int i = 0; i < targetServerCount; ++i) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
index 6a2cf33..ee410ca 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
@@ -31,7 +31,6 @@ import java.util.Optional;
 import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -757,8 +756,8 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase {
     if (util.isDistributedCluster()) {
       util.getConfiguration().setIfUnset(NUM_MAPS_KEY,
           Integer.toString(util.getAdmin()
-                               .getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
-                               .getServersSize() * 10)
+                               .getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+                               .getLiveServerMetrics().size() * 10)
       );
       util.getConfiguration().setIfUnset(NUM_IMPORT_ROUNDS_KEY, "5");
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
index d74a463..850e123 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
@@ -28,7 +28,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
 import org.apache.hadoop.hbase.ClusterStatus;
@@ -618,7 +617,8 @@ public class IntegrationTestMTTR {
     @Override
     public Boolean call() throws Exception {
       int colsPerKey = 10;
-      int numServers = util.getHBaseClusterInterface().getInitialClusterStatus().getServersSize();
+      int numServers = util.getHBaseClusterInterface().getInitialClusterMetrics()
+        .getLiveServerMetrics().size();
       int numKeys = numServers * 5000;
       int writeThreads = 10;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index 5ce0bba..ccdbdf0 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -36,7 +36,6 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.HelpFormatter;
@@ -82,11 +81,11 @@ import org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl;
 import org.apache.hadoop.hbase.mapreduce.WALPlayer;
 import org.apache.hadoop.hbase.regionserver.FlushAllLargeStoresPolicy;
 import org.apache.hadoop.hbase.regionserver.FlushPolicyFactory;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.RegionSplitter;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.NullWritable;
@@ -710,8 +709,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
           if (conf.getBoolean(HBaseTestingUtility.PRESPLIT_TEST_TABLE_KEY,
               HBaseTestingUtility.PRESPLIT_TEST_TABLE)) {
             int numberOfServers =
-                admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
-                    .getServers().size();
+                admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+                    .getLiveServerMetrics().size();
             if (numberOfServers == 0) {
               throw new IllegalStateException("No live regionservers");
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java
index 8986b9f..b5e99d2 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java
@@ -25,7 +25,6 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.IntegrationTestIngest;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
 import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
 import org.apache.hadoop.hbase.client.Admin;
@@ -43,6 +41,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
+import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.LoadTestTool;
 import org.apache.hadoop.hbase.util.MultiThreadedReader;
 import org.apache.hadoop.hbase.util.Threads;
@@ -142,7 +141,7 @@ public class IntegrationTestTimeBoundedRequestsWithRegionReplicas extends Integr
   protected void runIngestTest(long defaultRunTime, long keysPerServerPerIter, int colsPerKey,
       int recordSize, int writeThreads, int readThreads) throws Exception {
     LOG.info("Cluster size:"+
-      util.getHBaseClusterInterface().getClusterStatus().getServersSize());
+      util.getHBaseClusterInterface().getClusterMetrics().getLiveServerMetrics().size());
 
     long start = System.currentTimeMillis();
     String runtimeKey = String.format(RUN_TIME_KEY, this.getClass().getSimpleName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/654edc5f/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/RegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/RegionSizeCalculator.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/RegionSizeCalculator.java
index f50de5c..40cd34f 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/RegionSizeCalculator.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/RegionSizeCalculator.java
@@ -23,11 +23,11 @@ import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.RegionMetrics;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Size;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -79,12 +79,13 @@ public class RegionSizeCalculator {
     Set<ServerName> tableServers = getRegionServersOfTable(regionLocator);
 
     for (ServerName tableServerName : tableServers) {
-      Map<byte[], RegionLoad> regionLoads =
-          admin.getRegionLoad(tableServerName, regionLocator.getName());
-      for (RegionLoad regionLoad : regionLoads.values()) {
+      for (RegionMetrics regionLoad : admin.getRegionMetrics(
+        tableServerName,regionLocator.getName())) {
+
+        byte[] regionId = regionLoad.getRegionName();
+        long regionSizeBytes
+          = ((long) regionLoad.getStoreFileSize().get(Size.Unit.MEGABYTE)) * MEGABYTE;
 
-        byte[] regionId = regionLoad.getName();
-        long regionSizeBytes = regionLoad.getStorefileSizeMB() * MEGABYTE;
         sizeMap.put(regionId, regionSizeBytes);
 
         if (LOG.isDebugEnabled()) {