You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bh...@apache.org on 2020/09/19 22:05:03 UTC

[hbase] 07/09: HBASE-23305: Implement master based registry for client connections

This is an automated email from the ASF dual-hosted git repository.

bharathv pushed a commit to branch branch-1
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit ebe9e68274cddf483957c6e5f1b4b01e8491fb7e
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Thu Sep 3 19:30:13 2020 -0700

    HBASE-23305: Implement master based registry for client connections
    
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../hadoop/hbase/client/ConnectionManager.java     |    7 +-
 .../hadoop/hbase/client/ConnectionRegistry.java    |    9 +-
 .../hbase/client/ConnectionRegistryFactory.java    |    6 +-
 .../apache/hadoop/hbase/client/MasterRegistry.java |  233 ++++
 ...tionRegistry.java => ZKConnectionRegistry.java} |    8 +-
 .../apache/hadoop/hbase/security/SecurityInfo.java |    3 +
 .../hadoop/hbase/client/TestAsyncProcess.java      |    6 +-
 .../hadoop/hbase/client/TestClientNoCluster.java   |    4 +
 .../java/org/apache/hadoop/hbase/HConstants.java   |    3 +-
 .../exceptions/MasterRegistryFetchException.java   |   37 +-
 .../java/org/apache/hadoop/hbase/util/DNS.java     |   11 +
 .../apache/hadoop/hbase/util/PrettyPrinter.java    |   19 +
 .../hbase/protobuf/generated/MasterProtos.java     | 1218 +++++++++++++++++---
 hbase-protocol/src/main/protobuf/Master.proto      |   12 +
 .../org/apache/hadoop/hbase/master/HMaster.java    |   15 +
 .../hadoop/hbase/master/MasterRpcServices.java     |   13 +
 .../apache/hadoop/hbase/util/JVMClusterUtil.java   |    7 +
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |    3 +
 .../hadoop/hbase/client/TestFromClientSide.java    |   85 +-
 .../client/TestFromClientSideWithCoprocessor.java  |   29 +-
 .../hadoop/hbase/client/TestMasterRegistry.java    |  133 +++
 21 files changed, 1640 insertions(+), 221 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
index 9798e72..5addc7a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
@@ -942,7 +942,7 @@ class ConnectionManager {
 
     protected String clusterId = null;
 
-    void retrieveClusterId() {
+    void retrieveClusterId() throws IOException {
       if (clusterId != null) return;
       this.clusterId = this.registry.getClusterId();
       if (clusterId == null) {
@@ -1592,7 +1592,7 @@ class ConnectionManager {
       private Object makeStubNoRetries() throws IOException, ServiceException {
         ServerName sn = registry.getActiveMaster();
         if (sn == null) {
-          String msg = "ZooKeeper available but no active master location found";
+          String msg = "No active master location found";
           LOG.info(msg);
           throw new MasterNotRunningException(msg);
         }
@@ -2587,6 +2587,9 @@ class ConnectionManager {
       if (this.closed) {
         return;
       }
+      if (this.registry != null) {
+        this.registry.close();
+      }
       closeMaster();
       shutdownPools();
       if (this.metrics != null) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java
index 9c4f22a..353ff61 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java
@@ -33,7 +33,7 @@ interface ConnectionRegistry {
   /**
    * @param connection
    */
-  void init(Connection connection);
+  void init(Connection connection) throws IOException;
 
   /**
    * @return the currently active master, null if none exists.
@@ -49,11 +49,16 @@ interface ConnectionRegistry {
   /**
    * @return Cluster id.
    */
-  String getClusterId();
+  String getClusterId() throws IOException;
 
   /**
    * @return Count of 'running' regionservers
    * @throws IOException
    */
   int getCurrentNrHRS() throws IOException;
+
+  /**
+   * Cleanup state, if any.
+   */
+  void close();
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
index c166e21..eaef389 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 /**
@@ -26,7 +27,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
  */
 @InterfaceAudience.Private
 class ConnectionRegistryFactory {
-  static final String REGISTRY_IMPL_CONF_KEY = "hbase.client.registry.impl";
 
   /**
    * @return The cluster registry implementation to use.
@@ -34,8 +34,8 @@ class ConnectionRegistryFactory {
    */
   static ConnectionRegistry getRegistry(final Connection connection)
   throws IOException {
-    String registryClass = connection.getConfiguration().get(REGISTRY_IMPL_CONF_KEY,
-      ZooKeeperConnectionRegistry.class.getName());
+    String registryClass = connection.getConfiguration().get(HConstants.REGISTRY_IMPL_CONF_KEY,
+      ZKConnectionRegistry.class.getName());
     ConnectionRegistry registry = null;
     try {
       registry = (ConnectionRegistry)Class.forName(registryClass).getDeclaredConstructor().newInstance();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
new file mode 100644
index 0000000..882173f
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
@@ -0,0 +1,233 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.apache.hadoop.hbase.util.DNS.getMasterHostname;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.net.HostAndPort;
+import com.google.protobuf.Message;
+import com.google.protobuf.RpcController;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.MasterRegistryFetchException;
+import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.hadoop.hbase.ipc.RpcClientFactory;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.security.User;
+
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ClientMetaService;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse;
+
+/**
+ * Master based registry implementation. Makes RPCs to the configured master addresses from config
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}. All the registry methods are
+ * blocking unlike implementations in other branches.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements ConnectionRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  private ImmutableMap<String, ClientMetaService.Interface> masterAddr2Stub;
+
+  // RPC client used to talk to the masters.
+  private RpcClient rpcClient;
+  private RpcControllerFactory rpcControllerFactory;
+  private int rpcTimeoutMs;
+
+  @Override
+  public void init(Connection connection) throws IOException {
+    Configuration conf = connection.getConfiguration();
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+        conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    // HBASE-25051: we pass cluster id as null here since we do not have a cluster id yet, we have
+    // to fetch this through the master registry...
+    // This is a problem as we will use the cluster id to determine the authentication method
+    rpcClient = RpcClientFactory.createClient(conf, null);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+    populateMasterStubs(parseMasterAddrs(conf));
+  }
+
+  private interface Callable <T extends Message> {
+    T call(ClientMetaService.Interface stub, RpcController controller) throws IOException;
+  }
+
+  private <T extends Message> T doCall(Callable<T> callable) throws MasterRegistryFetchException {
+    Exception lastException = null;
+    Set<String> masters = masterAddr2Stub.keySet();
+    List<ClientMetaService.Interface> stubs = new ArrayList<>(masterAddr2Stub.values());
+    Collections.shuffle(stubs, ThreadLocalRandom.current());
+    for (ClientMetaService.Interface stub: stubs) {
+      HBaseRpcController controller = rpcControllerFactory.newController();
+      try {
+        T resp = callable.call(stub, controller);
+        if (controller.failed()) {
+          lastException = controller.getFailed();
+          continue;
+        }
+        return resp;
+      } catch (Exception e) {
+        lastException = e;
+      }
+    }
+    // rpcs to all masters failed.
+    throw new MasterRegistryFetchException(masters, lastException);
+  }
+
+  @Override
+  public ServerName getActiveMaster() throws IOException {
+    GetActiveMasterResponse resp = doCall(new Callable<GetActiveMasterResponse>() {
+      @Override
+      public GetActiveMasterResponse call(
+          ClientMetaService.Interface stub, RpcController controller) throws IOException {
+        BlockingRpcCallback<GetActiveMasterResponse> cb = new BlockingRpcCallback<>();
+        stub.getActiveMaster(controller, GetActiveMasterRequest.getDefaultInstance(), cb);
+        return cb.get();
+      }
+    });
+    if (!resp.hasServerName() || resp.getServerName() == null) {
+      throw new HBaseIOException("No active master found");
+    }
+    return ProtobufUtil.toServerName(resp.getServerName());
+  }
+
+  @Override
+  public RegionLocations getMetaRegionLocations() throws IOException {
+    GetMetaRegionLocationsResponse resp = doCall(new Callable<GetMetaRegionLocationsResponse>() {
+      @Override
+      public GetMetaRegionLocationsResponse call(
+          ClientMetaService.Interface stub, RpcController controller) throws IOException {
+        BlockingRpcCallback<GetMetaRegionLocationsResponse> cb = new BlockingRpcCallback<>();
+        stub.getMetaRegionLocations(controller, GetMetaRegionLocationsRequest.getDefaultInstance(),
+            cb);
+        return cb.get();
+      }
+    });
+    List<HRegionLocation> result = new ArrayList<>();
+    for (HBaseProtos.RegionLocation loc: resp.getMetaLocationsList()) {
+      result.add(ProtobufUtil.toRegionLocation(loc));
+    }
+    return new RegionLocations(result);
+  }
+
+  @Override
+  public String getClusterId() throws IOException {
+    GetClusterIdResponse resp = doCall(new Callable<GetClusterIdResponse>() {
+      @Override
+      public GetClusterIdResponse call(ClientMetaService.Interface stub, RpcController controller)
+          throws IOException {
+        BlockingRpcCallback<GetClusterIdResponse> cb = new BlockingRpcCallback<>();
+        stub.getClusterId(controller, GetClusterIdRequest.getDefaultInstance(), cb);
+        return cb.get();
+      }
+    });
+    return resp.getClusterId();
+  }
+
+  @Override
+  public int getCurrentNrHRS() throws IOException {
+    GetNumLiveRSResponse resp = doCall(new Callable<GetNumLiveRSResponse>() {
+      @Override
+      public GetNumLiveRSResponse call(ClientMetaService.Interface stub, RpcController controller)
+          throws IOException {
+        BlockingRpcCallback<GetNumLiveRSResponse> cb = new BlockingRpcCallback<>();
+        stub.getNumLiveRS(controller, GetNumLiveRSRequest.getDefaultInstance(), cb);
+        return cb.get();
+      }
+    });
+    return resp.getNumRegionServers();
+  }
+
+  @Override
+  public void close() {
+    if (rpcClient != null) {
+      rpcClient.close();
+    }
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is comma
+   * separated host[:port] values. If no port number if specified, default master port is assumed.
+   * @param conf Configuration to parse from.
+   */
+  @InterfaceAudience.Private
+   public static Set<ServerName> parseMasterAddrs(Configuration conf) throws UnknownHostException {
+    Set<ServerName> masterAddrs = new HashSet<>();
+    String configuredMasters = getMasterAddr(conf);
+    for (String masterAddr : configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
+      HostAndPort masterHostPort =
+          HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
+      masterAddrs.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
+    }
+    Preconditions.checkArgument(!masterAddrs.isEmpty(), "At least one master address is needed");
+    return masterAddrs;
+  }
+
+  /**
+   * Builds the default master address end point if it is not specified in the configuration.
+   * <p/>
+   * Will be called in {@code HBaseTestingUtility}.
+   */
+  @InterfaceAudience.Private
+  public static String getMasterAddr(Configuration conf) throws UnknownHostException {
+    String masterAddrFromConf = conf.get(HConstants.MASTER_ADDRS_KEY);
+    if (!Strings.isNullOrEmpty(masterAddrFromConf)) {
+      return masterAddrFromConf;
+    }
+    String hostname = getMasterHostname(conf);
+    int port = conf.getInt(HConstants.MASTER_PORT, HConstants.DEFAULT_MASTER_PORT);
+    return String.format("%s:%d", hostname, port);
+  }
+
+  void populateMasterStubs(Set<ServerName> masters) throws IOException {
+    Preconditions.checkNotNull(masters);
+    ImmutableMap.Builder<String, ClientMetaService.Interface> builder = ImmutableMap.builder();
+    User user = User.getCurrent();
+    for (ServerName masterAddr : masters) {
+      builder.put(masterAddr.toString(), ClientMetaService.newStub(
+          rpcClient.createRpcChannel(masterAddr, user, rpcTimeoutMs)));
+    }
+    masterAddr2Stub = builder.build();
+  }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
similarity index 96%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperConnectionRegistry.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
index 0401aee..c656da8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperConnectionRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
@@ -36,8 +36,8 @@ import org.apache.zookeeper.KeeperException;
 /**
  * A cluster registry that stores to zookeeper.
  */
-class ZooKeeperConnectionRegistry implements ConnectionRegistry {
-  private static final Log LOG = LogFactory.getLog(ZooKeeperConnectionRegistry.class);
+class ZKConnectionRegistry implements ConnectionRegistry {
+  private static final Log LOG = LogFactory.getLog(ZKConnectionRegistry.class);
   // Needs an instance of hci to function.  Set after construct this instance.
   ConnectionManager.HConnectionImplementation hci;
 
@@ -126,4 +126,8 @@ class ZooKeeperConnectionRegistry implements ConnectionRegistry {
       throw new IOException("Unexpected ZooKeeper exception", ke);
     }
   }
+
+  @Override
+  public void close() {
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java
index eca54a4..1091ee6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java
@@ -23,6 +23,7 @@ import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.TokenIdentifier.Kind;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ClientMetaService;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos;
@@ -42,6 +43,8 @@ public class SecurityInfo {
         new SecurityInfo("hbase.regionserver.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
     infos.put(MasterService.getDescriptor().getName(),
         new SecurityInfo("hbase.master.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
+    infos.put(ClientMetaService.getDescriptor().getName(),
+        new SecurityInfo("hbase.master.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
     infos.put(RegionServerStatusProtos.RegionServerStatusService.getDescriptor().getName(),
         new SecurityInfo("hbase.master.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
   }
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index d10cc2f..a73674a 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -483,6 +483,10 @@ public class TestAsyncProcess {
       public int getCurrentNrHRS() throws IOException {
         return 1;
       }
+
+      @Override
+      public void close() {
+      }
     }
 
     final AtomicInteger nbThreads = new AtomicInteger(0);
@@ -492,7 +496,7 @@ public class TestAsyncProcess {
     }
 
     private static Configuration setupConf(Configuration conf) {
-      conf.setClass(ConnectionRegistryFactory.REGISTRY_IMPL_CONF_KEY, TestConnectionRegistry.class, ConnectionRegistry.class);
+      conf.setClass(HConstants.REGISTRY_IMPL_CONF_KEY, TestConnectionRegistry.class, ConnectionRegistry.class);
       return conf;
     }
 
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
index d2e4f0f..5c85980 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
@@ -138,6 +138,10 @@ public class TestClientNoCluster extends Configured implements Tool {
     public int getCurrentNrHRS() throws IOException {
       return 1;
     }
+
+    @Override
+    public void close() {
+    }
   }
 
   /**
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 28b2d1c..41e7525 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -79,6 +79,7 @@ public final class HConstants {
     Bytes.SIZEOF_BYTE + 2 * Bytes.SIZEOF_INT;
   /** Just an array of bytes of the right size. */
   public static final byte[] HFILEBLOCK_DUMMY_HEADER = new byte[HFILEBLOCK_HEADER_SIZE];
+  public static final String REGISTRY_IMPL_CONF_KEY = "hbase.client.registry.impl";
 
   //End HFileBlockConstants.
 
@@ -176,7 +177,7 @@ public final class HConstants {
   public static final String MASTER_INFO_PORT = "hbase.master.info.port";
 
   /** Configuration key for the list of master host:ports **/
-  public static final String MASTER_ADDRS_KEY = "hbase.master.addrs";
+  public static final String MASTER_ADDRS_KEY = "hbase.masters";
 
   public static final String MASTER_ADDRS_DEFAULT =  "localhost:" + DEFAULT_MASTER_PORT;
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
similarity index 52%
copy from hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
copy to hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
index c166e21..3a66f61 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,34 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.client;
-
-import java.io.IOException;
+package org.apache.hadoop.hbase.exceptions;
 
+import java.util.Set;
+import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
 
 /**
- * Get instance of configured Connection Registry.
+ * Exception thrown when an master registry RPC fails in client. The exception includes the list of
+ * masters to which RPC was attempted and the last exception encountered. Prior exceptions are
+ * included in the logs.
  */
 @InterfaceAudience.Private
-class ConnectionRegistryFactory {
-  static final String REGISTRY_IMPL_CONF_KEY = "hbase.client.registry.impl";
+public class MasterRegistryFetchException extends HBaseIOException {
+
+  private static final long serialVersionUID = 6992134872168185171L;
 
-  /**
-   * @return The cluster registry implementation to use.
-   * @throws IOException
-   */
-  static ConnectionRegistry getRegistry(final Connection connection)
-  throws IOException {
-    String registryClass = connection.getConfiguration().get(REGISTRY_IMPL_CONF_KEY,
-      ZooKeeperConnectionRegistry.class.getName());
-    ConnectionRegistry registry = null;
-    try {
-      registry = (ConnectionRegistry)Class.forName(registryClass).getDeclaredConstructor().newInstance();
-    } catch (Throwable t) {
-      throw new IOException(t);
-    }
-    registry.init(connection);
-    return registry;
+  public MasterRegistryFetchException(Set<String> masters, Throwable failure) {
+    super(String.format("Exception making rpc to masters %s", PrettyPrinter.toString(masters)),
+        failure);
   }
 }
\ No newline at end of file
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DNS.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DNS.java
index 4b9e87f..1241dc8 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DNS.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DNS.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.util;
 import java.lang.reflect.Method;
 import java.net.UnknownHostException;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 /**
@@ -66,4 +67,14 @@ public final class DNS {
       return org.apache.hadoop.net.DNS.getDefaultHost(strInterface, nameserver);
     }
   }
+
+  public static String getMasterHostname(Configuration conf) throws UnknownHostException {
+    String hostname = conf.get("hbase.master.hostname", "");
+    if (hostname.isEmpty()) {
+      return Strings.domainNamePointerToHostName(getDefaultHost(
+          conf.get("hbase.master.dns.interface", "default"),
+          conf.get("hbase.master.dns.nameserver", "default")));
+    }
+    return hostname;
+  }
 }
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java
index efdd144..6530447 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java
@@ -19,6 +19,11 @@
 
 package org.apache.hadoop.hbase.util;
 
+import com.google.common.base.Joiner;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 
@@ -97,4 +102,18 @@ public class PrettyPrinter {
     return sb.toString();
   }
 
+  /**
+   * Pretty prints a collection of any type to a string. Relies on toString() implementation of the
+   * object type.
+   * @param collection collection to pretty print.
+   * @return Pretty printed string for the collection.
+   */
+  public static String toString(Collection<?> collection) {
+    List<String> stringList = new ArrayList<>();
+    for (Object o: collection) {
+      stringList.add(Objects.toString(o));
+    }
+    return "[" + Joiner.on(',').join(stringList) + "]";
+  }
+
 }
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
index 87b780b..909c144 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
@@ -68864,6 +68864,794 @@ public final class MasterProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.GetMetaRegionLocationsResponse)
   }
 
+  public interface GetNumLiveRSRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  /**
+   * Protobuf type {@code hbase.pb.GetNumLiveRSRequest}
+   *
+   * <pre>
+   ** Request and response to get the number of live region servers 
+   * </pre>
+   */
+  public static final class GetNumLiveRSRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements GetNumLiveRSRequestOrBuilder {
+    // Use GetNumLiveRSRequest.newBuilder() to construct.
+    private GetNumLiveRSRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private GetNumLiveRSRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final GetNumLiveRSRequest defaultInstance;
+    public static GetNumLiveRSRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public GetNumLiveRSRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GetNumLiveRSRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNumLiveRSRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNumLiveRSRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<GetNumLiveRSRequest> PARSER =
+        new com.google.protobuf.AbstractParser<GetNumLiveRSRequest>() {
+      public GetNumLiveRSRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new GetNumLiveRSRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<GetNumLiveRSRequest> getParserForType() {
+      return PARSER;
+    }
+
+    private void initFields() {
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest) obj;
+
+      boolean result = true;
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GetNumLiveRSRequest}
+     *
+     * <pre>
+     ** Request and response to get the number of live region servers 
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNumLiveRSRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNumLiveRSRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNumLiveRSRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest(this);
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest.getDefaultInstance()) return this;
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.GetNumLiveRSRequest)
+    }
+
+    static {
+      defaultInstance = new GetNumLiveRSRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.GetNumLiveRSRequest)
+  }
+
+  public interface GetNumLiveRSResponseOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required int32 num_region_servers = 1;
+    /**
+     * <code>required int32 num_region_servers = 1;</code>
+     */
+    boolean hasNumRegionServers();
+    /**
+     * <code>required int32 num_region_servers = 1;</code>
+     */
+    int getNumRegionServers();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.GetNumLiveRSResponse}
+   */
+  public static final class GetNumLiveRSResponse extends
+      com.google.protobuf.GeneratedMessage
+      implements GetNumLiveRSResponseOrBuilder {
+    // Use GetNumLiveRSResponse.newBuilder() to construct.
+    private GetNumLiveRSResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private GetNumLiveRSResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final GetNumLiveRSResponse defaultInstance;
+    public static GetNumLiveRSResponse getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public GetNumLiveRSResponse getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GetNumLiveRSResponse(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              numRegionServers_ = input.readInt32();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNumLiveRSResponse_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNumLiveRSResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<GetNumLiveRSResponse> PARSER =
+        new com.google.protobuf.AbstractParser<GetNumLiveRSResponse>() {
+      public GetNumLiveRSResponse parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new GetNumLiveRSResponse(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<GetNumLiveRSResponse> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required int32 num_region_servers = 1;
+    public static final int NUM_REGION_SERVERS_FIELD_NUMBER = 1;
+    private int numRegionServers_;
+    /**
+     * <code>required int32 num_region_servers = 1;</code>
+     */
+    public boolean hasNumRegionServers() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required int32 num_region_servers = 1;</code>
+     */
+    public int getNumRegionServers() {
+      return numRegionServers_;
+    }
+
+    private void initFields() {
+      numRegionServers_ = 0;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasNumRegionServers()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeInt32(1, numRegionServers_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(1, numRegionServers_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse) obj;
+
+      boolean result = true;
+      result = result && (hasNumRegionServers() == other.hasNumRegionServers());
+      if (hasNumRegionServers()) {
+        result = result && (getNumRegionServers()
+            == other.getNumRegionServers());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasNumRegionServers()) {
+        hash = (37 * hash) + NUM_REGION_SERVERS_FIELD_NUMBER;
+        hash = (53 * hash) + getNumRegionServers();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GetNumLiveRSResponse}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponseOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNumLiveRSResponse_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNumLiveRSResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        numRegionServers_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNumLiveRSResponse_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.numRegionServers_ = numRegionServers_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.getDefaultInstance()) return this;
+        if (other.hasNumRegionServers()) {
+          setNumRegionServers(other.getNumRegionServers());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasNumRegionServers()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required int32 num_region_servers = 1;
+      private int numRegionServers_ ;
+      /**
+       * <code>required int32 num_region_servers = 1;</code>
+       */
+      public boolean hasNumRegionServers() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required int32 num_region_servers = 1;</code>
+       */
+      public int getNumRegionServers() {
+        return numRegionServers_;
+      }
+      /**
+       * <code>required int32 num_region_servers = 1;</code>
+       */
+      public Builder setNumRegionServers(int value) {
+        bitField0_ |= 0x00000001;
+        numRegionServers_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required int32 num_region_servers = 1;</code>
+       */
+      public Builder clearNumRegionServers() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        numRegionServers_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.GetNumLiveRSResponse)
+    }
+
+    static {
+      defaultInstance = new GetNumLiveRSResponse(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.GetNumLiveRSResponse)
+  }
+
   /**
    * Protobuf service {@code hbase.pb.MasterService}
    */
@@ -74280,6 +75068,19 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest request,
           com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse> done);
 
+      /**
+       * <code>rpc GetNumLiveRS(.hbase.pb.GetNumLiveRSRequest) returns (.hbase.pb.GetNumLiveRSResponse);</code>
+       *
+       * <pre>
+       **
+       * Get number of live region servers.
+       * </pre>
+       */
+      public abstract void getNumLiveRS(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse> done);
+
     }
 
     public static com.google.protobuf.Service newReflectiveService(
@@ -74309,6 +75110,14 @@ public final class MasterProtos {
           impl.getMetaRegionLocations(controller, request, done);
         }
 
+        @java.lang.Override
+        public  void getNumLiveRS(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse> done) {
+          impl.getNumLiveRS(controller, request, done);
+        }
+
       };
     }
 
@@ -74337,6 +75146,8 @@ public final class MasterProtos {
               return impl.getActiveMaster(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest)request);
             case 2:
               return impl.getMetaRegionLocations(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest)request);
+            case 3:
+              return impl.getNumLiveRS(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -74357,6 +75168,8 @@ public final class MasterProtos {
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest.getDefaultInstance();
             case 2:
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest.getDefaultInstance();
+            case 3:
+              return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -74377,6 +75190,8 @@ public final class MasterProtos {
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.getDefaultInstance();
             case 2:
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse.getDefaultInstance();
+            case 3:
+              return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -74424,6 +75239,19 @@ public final class MasterProtos {
         org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest request,
         com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse> done);
 
+    /**
+     * <code>rpc GetNumLiveRS(.hbase.pb.GetNumLiveRSRequest) returns (.hbase.pb.GetNumLiveRSResponse);</code>
+     *
+     * <pre>
+     **
+     * Get number of live region servers.
+     * </pre>
+     */
+    public abstract void getNumLiveRS(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse> done);
+
     public static final
         com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
@@ -74461,6 +75289,11 @@ public final class MasterProtos {
             com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse>specializeCallback(
               done));
           return;
+        case 3:
+          this.getNumLiveRS(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse>specializeCallback(
+              done));
+          return;
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -74481,6 +75314,8 @@ public final class MasterProtos {
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest.getDefaultInstance();
         case 2:
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest.getDefaultInstance();
+        case 3:
+          return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -74501,6 +75336,8 @@ public final class MasterProtos {
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.getDefaultInstance();
         case 2:
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse.getDefaultInstance();
+        case 3:
+          return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -74566,6 +75403,21 @@ public final class MasterProtos {
             org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse.class,
             org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse.getDefaultInstance()));
       }
+
+      public  void getNumLiveRS(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(3),
+          controller,
+          request,
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.class,
+            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.getDefaultInstance()));
+      }
     }
 
     public static BlockingInterface newBlockingStub(
@@ -74588,6 +75440,11 @@ public final class MasterProtos {
           com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest request)
           throws com.google.protobuf.ServiceException;
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse getNumLiveRS(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest request)
+          throws com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
@@ -74632,6 +75489,18 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse.getDefaultInstance());
       }
 
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse getNumLiveRS(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(3),
+          controller,
+          request,
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse.getDefaultInstance());
+      }
+
     }
 
     // @@protoc_insertion_point(class_scope:hbase.pb.ClientMetaService)
@@ -75292,6 +76161,16 @@ public final class MasterProtos {
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_GetMetaRegionLocationsResponse_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_GetNumLiveRSRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_GetNumLiveRSRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_GetNumLiveRSResponse_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_GetNumLiveRSResponse_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -75516,168 +76395,171 @@ public final class MasterProtos {
       "r_name\030\001 \001(\0132\024.hbase.pb.ServerName\"\037\n\035Ge" +
       "tMetaRegionLocationsRequest\"R\n\036GetMetaRe" +
       "gionLocationsResponse\0220\n\016meta_locations\030" +
-      "\001 \003(\0132\030.hbase.pb.RegionLocation*(\n\020Maste" +
-      "rSwitchType\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\363.\n\rMa" +
-      "sterService\022e\n\024GetSchemaAlterStatus\022%.hb" +
-      "ase.pb.GetSchemaAlterStatusRequest\032&.hba" +
-      "se.pb.GetSchemaAlterStatusResponse\022b\n\023Ge",
-      "tTableDescriptors\022$.hbase.pb.GetTableDes" +
-      "criptorsRequest\032%.hbase.pb.GetTableDescr" +
-      "iptorsResponse\022P\n\rGetTableNames\022\036.hbase." +
-      "pb.GetTableNamesRequest\032\037.hbase.pb.GetTa" +
-      "bleNamesResponse\022Y\n\020GetClusterStatus\022!.h" +
-      "base.pb.GetClusterStatusRequest\032\".hbase." +
-      "pb.GetClusterStatusResponse\022V\n\017IsMasterR" +
-      "unning\022 .hbase.pb.IsMasterRunningRequest" +
-      "\032!.hbase.pb.IsMasterRunningResponse\022D\n\tA" +
-      "ddColumn\022\032.hbase.pb.AddColumnRequest\032\033.h",
-      "base.pb.AddColumnResponse\022M\n\014DeleteColum" +
-      "n\022\035.hbase.pb.DeleteColumnRequest\032\036.hbase" +
-      ".pb.DeleteColumnResponse\022M\n\014ModifyColumn" +
-      "\022\035.hbase.pb.ModifyColumnRequest\032\036.hbase." +
-      "pb.ModifyColumnResponse\022G\n\nMoveRegion\022\033." +
-      "hbase.pb.MoveRegionRequest\032\034.hbase.pb.Mo" +
-      "veRegionResponse\022k\n\026DispatchMergingRegio" +
-      "ns\022\'.hbase.pb.DispatchMergingRegionsRequ" +
-      "est\032(.hbase.pb.DispatchMergingRegionsRes" +
-      "ponse\022M\n\014AssignRegion\022\035.hbase.pb.AssignR",
-      "egionRequest\032\036.hbase.pb.AssignRegionResp" +
-      "onse\022S\n\016UnassignRegion\022\037.hbase.pb.Unassi" +
-      "gnRegionRequest\032 .hbase.pb.UnassignRegio" +
-      "nResponse\022P\n\rOfflineRegion\022\036.hbase.pb.Of" +
-      "flineRegionRequest\032\037.hbase.pb.OfflineReg" +
-      "ionResponse\022J\n\013DeleteTable\022\034.hbase.pb.De" +
-      "leteTableRequest\032\035.hbase.pb.DeleteTableR" +
-      "esponse\022P\n\rtruncateTable\022\036.hbase.pb.Trun" +
-      "cateTableRequest\032\037.hbase.pb.TruncateTabl" +
-      "eResponse\022J\n\013EnableTable\022\034.hbase.pb.Enab",
-      "leTableRequest\032\035.hbase.pb.EnableTableRes" +
-      "ponse\022M\n\014DisableTable\022\035.hbase.pb.Disable" +
-      "TableRequest\032\036.hbase.pb.DisableTableResp" +
-      "onse\022J\n\013ModifyTable\022\034.hbase.pb.ModifyTab" +
-      "leRequest\032\035.hbase.pb.ModifyTableResponse" +
-      "\022J\n\013CreateTable\022\034.hbase.pb.CreateTableRe" +
-      "quest\032\035.hbase.pb.CreateTableResponse\022A\n\010" +
-      "Shutdown\022\031.hbase.pb.ShutdownRequest\032\032.hb" +
-      "ase.pb.ShutdownResponse\022G\n\nStopMaster\022\033." +
-      "hbase.pb.StopMasterRequest\032\034.hbase.pb.St",
-      "opMasterResponse\022h\n\031IsMasterInMaintenanc" +
-      "eMode\022$.hbase.pb.IsInMaintenanceModeRequ" +
-      "est\032%.hbase.pb.IsInMaintenanceModeRespon" +
-      "se\022>\n\007Balance\022\030.hbase.pb.BalanceRequest\032" +
-      "\031.hbase.pb.BalanceResponse\022_\n\022SetBalance" +
-      "rRunning\022#.hbase.pb.SetBalancerRunningRe" +
-      "quest\032$.hbase.pb.SetBalancerRunningRespo" +
-      "nse\022\\\n\021IsBalancerEnabled\022\".hbase.pb.IsBa" +
-      "lancerEnabledRequest\032#.hbase.pb.IsBalanc" +
-      "erEnabledResponse\022k\n\026SetSplitOrMergeEnab",
-      "led\022\'.hbase.pb.SetSplitOrMergeEnabledReq" +
-      "uest\032(.hbase.pb.SetSplitOrMergeEnabledRe" +
-      "sponse\022h\n\025IsSplitOrMergeEnabled\022&.hbase." +
-      "pb.IsSplitOrMergeEnabledRequest\032\'.hbase." +
-      "pb.IsSplitOrMergeEnabledResponse\022D\n\tNorm" +
-      "alize\022\032.hbase.pb.NormalizeRequest\032\033.hbas" +
-      "e.pb.NormalizeResponse\022e\n\024SetNormalizerR" +
-      "unning\022%.hbase.pb.SetNormalizerRunningRe" +
-      "quest\032&.hbase.pb.SetNormalizerRunningRes" +
-      "ponse\022b\n\023IsNormalizerEnabled\022$.hbase.pb.",
-      "IsNormalizerEnabledRequest\032%.hbase.pb.Is" +
-      "NormalizerEnabledResponse\022S\n\016RunCatalogS" +
-      "can\022\037.hbase.pb.RunCatalogScanRequest\032 .h" +
-      "base.pb.RunCatalogScanResponse\022e\n\024Enable" +
-      "CatalogJanitor\022%.hbase.pb.EnableCatalogJ" +
-      "anitorRequest\032&.hbase.pb.EnableCatalogJa" +
-      "nitorResponse\022n\n\027IsCatalogJanitorEnabled" +
-      "\022(.hbase.pb.IsCatalogJanitorEnabledReque" +
-      "st\032).hbase.pb.IsCatalogJanitorEnabledRes" +
-      "ponse\022V\n\017RunCleanerChore\022 .hbase.pb.RunC",
-      "leanerChoreRequest\032!.hbase.pb.RunCleaner" +
-      "ChoreResponse\022k\n\026SetCleanerChoreRunning\022" +
-      "\'.hbase.pb.SetCleanerChoreRunningRequest" +
-      "\032(.hbase.pb.SetCleanerChoreRunningRespon" +
-      "se\022h\n\025IsCleanerChoreEnabled\022&.hbase.pb.I" +
-      "sCleanerChoreEnabledRequest\032\'.hbase.pb.I" +
-      "sCleanerChoreEnabledResponse\022^\n\021ExecMast" +
-      "erService\022#.hbase.pb.CoprocessorServiceR" +
-      "equest\032$.hbase.pb.CoprocessorServiceResp" +
-      "onse\022A\n\010Snapshot\022\031.hbase.pb.SnapshotRequ",
-      "est\032\032.hbase.pb.SnapshotResponse\022h\n\025GetCo" +
-      "mpletedSnapshots\022&.hbase.pb.GetCompleted" +
-      "SnapshotsRequest\032\'.hbase.pb.GetCompleted" +
-      "SnapshotsResponse\022S\n\016DeleteSnapshot\022\037.hb" +
-      "ase.pb.DeleteSnapshotRequest\032 .hbase.pb." +
-      "DeleteSnapshotResponse\022S\n\016IsSnapshotDone" +
-      "\022\037.hbase.pb.IsSnapshotDoneRequest\032 .hbas" +
-      "e.pb.IsSnapshotDoneResponse\022V\n\017RestoreSn" +
-      "apshot\022 .hbase.pb.RestoreSnapshotRequest" +
-      "\032!.hbase.pb.RestoreSnapshotResponse\022h\n\025I",
-      "sRestoreSnapshotDone\022&.hbase.pb.IsRestor" +
-      "eSnapshotDoneRequest\032\'.hbase.pb.IsRestor" +
-      "eSnapshotDoneResponse\022P\n\rExecProcedure\022\036" +
-      ".hbase.pb.ExecProcedureRequest\032\037.hbase.p" +
-      "b.ExecProcedureResponse\022W\n\024ExecProcedure" +
-      "WithRet\022\036.hbase.pb.ExecProcedureRequest\032" +
-      "\037.hbase.pb.ExecProcedureResponse\022V\n\017IsPr" +
-      "ocedureDone\022 .hbase.pb.IsProcedureDoneRe" +
-      "quest\032!.hbase.pb.IsProcedureDoneResponse" +
-      "\022V\n\017ModifyNamespace\022 .hbase.pb.ModifyNam",
-      "espaceRequest\032!.hbase.pb.ModifyNamespace" +
-      "Response\022V\n\017CreateNamespace\022 .hbase.pb.C" +
-      "reateNamespaceRequest\032!.hbase.pb.CreateN" +
-      "amespaceResponse\022V\n\017DeleteNamespace\022 .hb" +
-      "ase.pb.DeleteNamespaceRequest\032!.hbase.pb" +
-      ".DeleteNamespaceResponse\022k\n\026GetNamespace" +
-      "Descriptor\022\'.hbase.pb.GetNamespaceDescri" +
-      "ptorRequest\032(.hbase.pb.GetNamespaceDescr" +
-      "iptorResponse\022q\n\030ListNamespaceDescriptor" +
-      "s\022).hbase.pb.ListNamespaceDescriptorsReq",
-      "uest\032*.hbase.pb.ListNamespaceDescriptors" +
-      "Response\022\206\001\n\037ListTableDescriptorsByNames" +
-      "pace\0220.hbase.pb.ListTableDescriptorsByNa" +
-      "mespaceRequest\0321.hbase.pb.ListTableDescr" +
-      "iptorsByNamespaceResponse\022t\n\031ListTableNa" +
-      "mesByNamespace\022*.hbase.pb.ListTableNames" +
-      "ByNamespaceRequest\032+.hbase.pb.ListTableN" +
-      "amesByNamespaceResponse\022A\n\010SetQuota\022\031.hb" +
-      "ase.pb.SetQuotaRequest\032\032.hbase.pb.SetQuo" +
-      "taResponse\022x\n\037getLastMajorCompactionTime",
-      "stamp\022).hbase.pb.MajorCompactionTimestam" +
-      "pRequest\032*.hbase.pb.MajorCompactionTimes" +
-      "tampResponse\022\212\001\n(getLastMajorCompactionT" +
-      "imestampForRegion\0222.hbase.pb.MajorCompac" +
-      "tionTimestampForRegionRequest\032*.hbase.pb" +
-      ".MajorCompactionTimestampResponse\022_\n\022get" +
-      "ProcedureResult\022#.hbase.pb.GetProcedureR" +
-      "esultRequest\032$.hbase.pb.GetProcedureResu" +
-      "ltResponse\022h\n\027getSecurityCapabilities\022%." +
-      "hbase.pb.SecurityCapabilitiesRequest\032&.h",
-      "base.pb.SecurityCapabilitiesResponse\022S\n\016" +
-      "AbortProcedure\022\037.hbase.pb.AbortProcedure" +
-      "Request\032 .hbase.pb.AbortProcedureRespons" +
-      "e\022S\n\016ListProcedures\022\037.hbase.pb.ListProce" +
-      "duresRequest\032 .hbase.pb.ListProceduresRe" +
-      "sponse\022Y\n\020ClearDeadServers\022!.hbase.pb.Cl" +
-      "earDeadServersRequest\032\".hbase.pb.ClearDe" +
-      "adServersResponse\022S\n\016ListNamespaces\022\037.hb" +
-      "ase.pb.ListNamespacesRequest\032 .hbase.pb." +
-      "ListNamespacesResponse\022b\n\025SwitchSnapshot",
-      "Cleanup\022#.hbase.pb.SetSnapshotCleanupReq" +
-      "uest\032$.hbase.pb.SetSnapshotCleanupRespon" +
-      "se\022q\n\030IsSnapshotCleanupEnabled\022).hbase.p" +
-      "b.IsSnapshotCleanupEnabledRequest\032*.hbas" +
-      "e.pb.IsSnapshotCleanupEnabledResponse\022P\n" +
-      "\rGetTableState\022\036.hbase.pb.GetTableStateR" +
-      "equest\032\037.hbase.pb.GetTableStateResponse2" +
-      "\247\002\n\021ClientMetaService\022M\n\014GetClusterId\022\035." +
-      "hbase.pb.GetClusterIdRequest\032\036.hbase.pb." +
-      "GetClusterIdResponse\022V\n\017GetActiveMaster\022",
-      " .hbase.pb.GetActiveMasterRequest\032!.hbas" +
-      "e.pb.GetActiveMasterResponse\022k\n\026GetMetaR" +
-      "egionLocations\022\'.hbase.pb.GetMetaRegionL" +
-      "ocationsRequest\032(.hbase.pb.GetMetaRegion" +
-      "LocationsResponseBB\n*org.apache.hadoop.h" +
-      "base.protobuf.generatedB\014MasterProtosH\001\210" +
-      "\001\001\240\001\001"
+      "\001 \003(\0132\030.hbase.pb.RegionLocation\"\025\n\023GetNu" +
+      "mLiveRSRequest\"2\n\024GetNumLiveRSResponse\022\032" +
+      "\n\022num_region_servers\030\001 \002(\005*(\n\020MasterSwit" +
+      "chType\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\363.\n\rMasterS" +
+      "ervice\022e\n\024GetSchemaAlterStatus\022%.hbase.p",
+      "b.GetSchemaAlterStatusRequest\032&.hbase.pb" +
+      ".GetSchemaAlterStatusResponse\022b\n\023GetTabl" +
+      "eDescriptors\022$.hbase.pb.GetTableDescript" +
+      "orsRequest\032%.hbase.pb.GetTableDescriptor" +
+      "sResponse\022P\n\rGetTableNames\022\036.hbase.pb.Ge" +
+      "tTableNamesRequest\032\037.hbase.pb.GetTableNa" +
+      "mesResponse\022Y\n\020GetClusterStatus\022!.hbase." +
+      "pb.GetClusterStatusRequest\032\".hbase.pb.Ge" +
+      "tClusterStatusResponse\022V\n\017IsMasterRunnin" +
+      "g\022 .hbase.pb.IsMasterRunningRequest\032!.hb",
+      "ase.pb.IsMasterRunningResponse\022D\n\tAddCol" +
+      "umn\022\032.hbase.pb.AddColumnRequest\032\033.hbase." +
+      "pb.AddColumnResponse\022M\n\014DeleteColumn\022\035.h" +
+      "base.pb.DeleteColumnRequest\032\036.hbase.pb.D" +
+      "eleteColumnResponse\022M\n\014ModifyColumn\022\035.hb" +
+      "ase.pb.ModifyColumnRequest\032\036.hbase.pb.Mo" +
+      "difyColumnResponse\022G\n\nMoveRegion\022\033.hbase" +
+      ".pb.MoveRegionRequest\032\034.hbase.pb.MoveReg" +
+      "ionResponse\022k\n\026DispatchMergingRegions\022\'." +
+      "hbase.pb.DispatchMergingRegionsRequest\032(",
+      ".hbase.pb.DispatchMergingRegionsResponse" +
+      "\022M\n\014AssignRegion\022\035.hbase.pb.AssignRegion" +
+      "Request\032\036.hbase.pb.AssignRegionResponse\022" +
+      "S\n\016UnassignRegion\022\037.hbase.pb.UnassignReg" +
+      "ionRequest\032 .hbase.pb.UnassignRegionResp" +
+      "onse\022P\n\rOfflineRegion\022\036.hbase.pb.Offline" +
+      "RegionRequest\032\037.hbase.pb.OfflineRegionRe" +
+      "sponse\022J\n\013DeleteTable\022\034.hbase.pb.DeleteT" +
+      "ableRequest\032\035.hbase.pb.DeleteTableRespon" +
+      "se\022P\n\rtruncateTable\022\036.hbase.pb.TruncateT",
+      "ableRequest\032\037.hbase.pb.TruncateTableResp" +
+      "onse\022J\n\013EnableTable\022\034.hbase.pb.EnableTab" +
+      "leRequest\032\035.hbase.pb.EnableTableResponse" +
+      "\022M\n\014DisableTable\022\035.hbase.pb.DisableTable" +
+      "Request\032\036.hbase.pb.DisableTableResponse\022" +
+      "J\n\013ModifyTable\022\034.hbase.pb.ModifyTableReq" +
+      "uest\032\035.hbase.pb.ModifyTableResponse\022J\n\013C" +
+      "reateTable\022\034.hbase.pb.CreateTableRequest" +
+      "\032\035.hbase.pb.CreateTableResponse\022A\n\010Shutd" +
+      "own\022\031.hbase.pb.ShutdownRequest\032\032.hbase.p",
+      "b.ShutdownResponse\022G\n\nStopMaster\022\033.hbase" +
+      ".pb.StopMasterRequest\032\034.hbase.pb.StopMas" +
+      "terResponse\022h\n\031IsMasterInMaintenanceMode" +
+      "\022$.hbase.pb.IsInMaintenanceModeRequest\032%" +
+      ".hbase.pb.IsInMaintenanceModeResponse\022>\n" +
+      "\007Balance\022\030.hbase.pb.BalanceRequest\032\031.hba" +
+      "se.pb.BalanceResponse\022_\n\022SetBalancerRunn" +
+      "ing\022#.hbase.pb.SetBalancerRunningRequest" +
+      "\032$.hbase.pb.SetBalancerRunningResponse\022\\" +
+      "\n\021IsBalancerEnabled\022\".hbase.pb.IsBalance",
+      "rEnabledRequest\032#.hbase.pb.IsBalancerEna" +
+      "bledResponse\022k\n\026SetSplitOrMergeEnabled\022\'" +
+      ".hbase.pb.SetSplitOrMergeEnabledRequest\032" +
+      "(.hbase.pb.SetSplitOrMergeEnabledRespons" +
+      "e\022h\n\025IsSplitOrMergeEnabled\022&.hbase.pb.Is" +
+      "SplitOrMergeEnabledRequest\032\'.hbase.pb.Is" +
+      "SplitOrMergeEnabledResponse\022D\n\tNormalize" +
+      "\022\032.hbase.pb.NormalizeRequest\032\033.hbase.pb." +
+      "NormalizeResponse\022e\n\024SetNormalizerRunnin" +
+      "g\022%.hbase.pb.SetNormalizerRunningRequest",
+      "\032&.hbase.pb.SetNormalizerRunningResponse" +
+      "\022b\n\023IsNormalizerEnabled\022$.hbase.pb.IsNor" +
+      "malizerEnabledRequest\032%.hbase.pb.IsNorma" +
+      "lizerEnabledResponse\022S\n\016RunCatalogScan\022\037" +
+      ".hbase.pb.RunCatalogScanRequest\032 .hbase." +
+      "pb.RunCatalogScanResponse\022e\n\024EnableCatal" +
+      "ogJanitor\022%.hbase.pb.EnableCatalogJanito" +
+      "rRequest\032&.hbase.pb.EnableCatalogJanitor" +
+      "Response\022n\n\027IsCatalogJanitorEnabled\022(.hb" +
+      "ase.pb.IsCatalogJanitorEnabledRequest\032).",
+      "hbase.pb.IsCatalogJanitorEnabledResponse" +
+      "\022V\n\017RunCleanerChore\022 .hbase.pb.RunCleane" +
+      "rChoreRequest\032!.hbase.pb.RunCleanerChore" +
+      "Response\022k\n\026SetCleanerChoreRunning\022\'.hba" +
+      "se.pb.SetCleanerChoreRunningRequest\032(.hb" +
+      "ase.pb.SetCleanerChoreRunningResponse\022h\n" +
+      "\025IsCleanerChoreEnabled\022&.hbase.pb.IsClea" +
+      "nerChoreEnabledRequest\032\'.hbase.pb.IsClea" +
+      "nerChoreEnabledResponse\022^\n\021ExecMasterSer" +
+      "vice\022#.hbase.pb.CoprocessorServiceReques",
+      "t\032$.hbase.pb.CoprocessorServiceResponse\022" +
+      "A\n\010Snapshot\022\031.hbase.pb.SnapshotRequest\032\032" +
+      ".hbase.pb.SnapshotResponse\022h\n\025GetComplet" +
+      "edSnapshots\022&.hbase.pb.GetCompletedSnaps" +
+      "hotsRequest\032\'.hbase.pb.GetCompletedSnaps" +
+      "hotsResponse\022S\n\016DeleteSnapshot\022\037.hbase.p" +
+      "b.DeleteSnapshotRequest\032 .hbase.pb.Delet" +
+      "eSnapshotResponse\022S\n\016IsSnapshotDone\022\037.hb" +
+      "ase.pb.IsSnapshotDoneRequest\032 .hbase.pb." +
+      "IsSnapshotDoneResponse\022V\n\017RestoreSnapsho",
+      "t\022 .hbase.pb.RestoreSnapshotRequest\032!.hb" +
+      "ase.pb.RestoreSnapshotResponse\022h\n\025IsRest" +
+      "oreSnapshotDone\022&.hbase.pb.IsRestoreSnap" +
+      "shotDoneRequest\032\'.hbase.pb.IsRestoreSnap" +
+      "shotDoneResponse\022P\n\rExecProcedure\022\036.hbas" +
+      "e.pb.ExecProcedureRequest\032\037.hbase.pb.Exe" +
+      "cProcedureResponse\022W\n\024ExecProcedureWithR" +
+      "et\022\036.hbase.pb.ExecProcedureRequest\032\037.hba" +
+      "se.pb.ExecProcedureResponse\022V\n\017IsProcedu" +
+      "reDone\022 .hbase.pb.IsProcedureDoneRequest",
+      "\032!.hbase.pb.IsProcedureDoneResponse\022V\n\017M" +
+      "odifyNamespace\022 .hbase.pb.ModifyNamespac" +
+      "eRequest\032!.hbase.pb.ModifyNamespaceRespo" +
+      "nse\022V\n\017CreateNamespace\022 .hbase.pb.Create" +
+      "NamespaceRequest\032!.hbase.pb.CreateNamesp" +
+      "aceResponse\022V\n\017DeleteNamespace\022 .hbase.p" +
+      "b.DeleteNamespaceRequest\032!.hbase.pb.Dele" +
+      "teNamespaceResponse\022k\n\026GetNamespaceDescr" +
+      "iptor\022\'.hbase.pb.GetNamespaceDescriptorR" +
+      "equest\032(.hbase.pb.GetNamespaceDescriptor",
+      "Response\022q\n\030ListNamespaceDescriptors\022).h" +
+      "base.pb.ListNamespaceDescriptorsRequest\032" +
+      "*.hbase.pb.ListNamespaceDescriptorsRespo" +
+      "nse\022\206\001\n\037ListTableDescriptorsByNamespace\022" +
+      "0.hbase.pb.ListTableDescriptorsByNamespa" +
+      "ceRequest\0321.hbase.pb.ListTableDescriptor" +
+      "sByNamespaceResponse\022t\n\031ListTableNamesBy" +
+      "Namespace\022*.hbase.pb.ListTableNamesByNam" +
+      "espaceRequest\032+.hbase.pb.ListTableNamesB" +
+      "yNamespaceResponse\022A\n\010SetQuota\022\031.hbase.p",
+      "b.SetQuotaRequest\032\032.hbase.pb.SetQuotaRes" +
+      "ponse\022x\n\037getLastMajorCompactionTimestamp" +
+      "\022).hbase.pb.MajorCompactionTimestampRequ" +
+      "est\032*.hbase.pb.MajorCompactionTimestampR" +
+      "esponse\022\212\001\n(getLastMajorCompactionTimest" +
+      "ampForRegion\0222.hbase.pb.MajorCompactionT" +
+      "imestampForRegionRequest\032*.hbase.pb.Majo" +
+      "rCompactionTimestampResponse\022_\n\022getProce" +
+      "dureResult\022#.hbase.pb.GetProcedureResult" +
+      "Request\032$.hbase.pb.GetProcedureResultRes",
+      "ponse\022h\n\027getSecurityCapabilities\022%.hbase" +
+      ".pb.SecurityCapabilitiesRequest\032&.hbase." +
+      "pb.SecurityCapabilitiesResponse\022S\n\016Abort" +
+      "Procedure\022\037.hbase.pb.AbortProcedureReque" +
+      "st\032 .hbase.pb.AbortProcedureResponse\022S\n\016" +
+      "ListProcedures\022\037.hbase.pb.ListProcedures" +
+      "Request\032 .hbase.pb.ListProceduresRespons" +
+      "e\022Y\n\020ClearDeadServers\022!.hbase.pb.ClearDe" +
+      "adServersRequest\032\".hbase.pb.ClearDeadSer" +
+      "versResponse\022S\n\016ListNamespaces\022\037.hbase.p",
+      "b.ListNamespacesRequest\032 .hbase.pb.ListN" +
+      "amespacesResponse\022b\n\025SwitchSnapshotClean" +
+      "up\022#.hbase.pb.SetSnapshotCleanupRequest\032" +
+      "$.hbase.pb.SetSnapshotCleanupResponse\022q\n" +
+      "\030IsSnapshotCleanupEnabled\022).hbase.pb.IsS" +
+      "napshotCleanupEnabledRequest\032*.hbase.pb." +
+      "IsSnapshotCleanupEnabledResponse\022P\n\rGetT" +
+      "ableState\022\036.hbase.pb.GetTableStateReques" +
+      "t\032\037.hbase.pb.GetTableStateResponse2\366\002\n\021C" +
+      "lientMetaService\022M\n\014GetClusterId\022\035.hbase",
+      ".pb.GetClusterIdRequest\032\036.hbase.pb.GetCl" +
+      "usterIdResponse\022V\n\017GetActiveMaster\022 .hba" +
+      "se.pb.GetActiveMasterRequest\032!.hbase.pb." +
+      "GetActiveMasterResponse\022k\n\026GetMetaRegion" +
+      "Locations\022\'.hbase.pb.GetMetaRegionLocati" +
+      "onsRequest\032(.hbase.pb.GetMetaRegionLocat" +
+      "ionsResponse\022M\n\014GetNumLiveRS\022\035.hbase.pb." +
+      "GetNumLiveRSRequest\032\036.hbase.pb.GetNumLiv" +
+      "eRSResponseBB\n*org.apache.hadoop.hbase.p" +
+      "rotobuf.generatedB\014MasterProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -76470,6 +77352,18 @@ public final class MasterProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetMetaRegionLocationsResponse_descriptor,
               new java.lang.String[] { "MetaLocations", });
+          internal_static_hbase_pb_GetNumLiveRSRequest_descriptor =
+            getDescriptor().getMessageTypes().get(131);
+          internal_static_hbase_pb_GetNumLiveRSRequest_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_GetNumLiveRSRequest_descriptor,
+              new java.lang.String[] { });
+          internal_static_hbase_pb_GetNumLiveRSResponse_descriptor =
+            getDescriptor().getMessageTypes().get(132);
+          internal_static_hbase_pb_GetNumLiveRSResponse_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_GetNumLiveRSResponse_descriptor,
+              new java.lang.String[] { "NumRegionServers", });
           return null;
         }
       };
diff --git a/hbase-protocol/src/main/protobuf/Master.proto b/hbase-protocol/src/main/protobuf/Master.proto
index b2fd3f8..bf16dd5 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -938,6 +938,13 @@ message GetMetaRegionLocationsResponse {
   repeated RegionLocation meta_locations = 1;
 }
 
+/** Request and response to get the number of live region servers */
+message GetNumLiveRSRequest {
+}
+message GetNumLiveRSResponse {
+  required int32 num_region_servers = 1;
+}
+
 /**
  * Implements all the RPCs needed by clients to look up cluster meta information needed for connection establishment.
  */
@@ -956,4 +963,9 @@ service ClientMetaService {
    * Get current meta replicas' region locations.
    */
   rpc GetMetaRegionLocations(GetMetaRegionLocationsRequest) returns(GetMetaRegionLocationsResponse);
+
+  /**
+   * Get number of live region servers.
+   */
+  rpc GetNumLiveRS(GetNumLiveRSRequest) returns(GetNumLiveRSResponse);
 }
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 1e03d44..2bb4362 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
@@ -2641,6 +2641,21 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     return masterFinishedInitializationTime;
   }
 
+  /**
+   * @return number of live region servers tracked by this master.
+   * @throws KeeperException if there is an issue with zookeeper connection.
+   */
+  public int getNumLiveRegionServers() throws KeeperException {
+    if (isActiveMaster()) {
+      return regionServerTracker.getOnlineServers().size();
+    }
+    // If the master is not active, we fall back to ZK to fetch the number of live region servers.
+    // This is an extra hop but that is okay since the ConnectionRegistry call that is serviced by
+    // this method is already deprecated and is not used in any active code paths. This method is
+    // here to only for the test code.
+    return ZKUtil.getNumberOfChildren(zooKeeper, zooKeeper.rsZNode);
+  }
+
   public int getNumWALFiles() {
     return procedureStore != null ? procedureStore.getActiveLogs().size() : 0;
   }
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 4af4560..e4eb654 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
@@ -109,6 +109,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLoca
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
@@ -1824,4 +1826,15 @@ public class MasterRpcServices extends RSRpcServices
     return response.build();
   }
 
+  @Override
+  public GetNumLiveRSResponse getNumLiveRS(RpcController rpcController, GetNumLiveRSRequest request)
+      throws ServiceException {
+    GetNumLiveRSResponse.Builder response = GetNumLiveRSResponse.newBuilder();
+    try {
+      response.setNumRegionServers(master.getNumLiveRegionServers());
+    } catch (KeeperException ke) {
+      throw new ServiceException(ke);
+    }
+    return response.build();
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
index 23c4f3c..4128c8c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import com.google.common.base.Preconditions;
 import java.io.InterruptedIOException;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
@@ -26,6 +27,7 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
@@ -144,6 +146,11 @@ public class JVMClusterUtil {
     } catch (Exception e) {
       throw new IOException(e);
     }
+    // Needed if a master based registry is configured for internal cluster connections. Here, we
+    // just add the current master host port since we do not know other master addresses up front
+    // in mini cluster tests.
+    c.set(HConstants.MASTER_ADDRS_KEY,
+        Preconditions.checkNotNull(server.getServerName().getAddress()).toString());
     return new JVMClusterUtil.MasterThread(server, index);
   }
 
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 a37c55d..a9b5fb1 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
@@ -1110,6 +1110,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     Configuration c = new Configuration(this.conf);
     this.hbaseCluster =
         new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass);
+    // Populate the master address configuration from mini cluster configuration.
+    conf.set(HConstants.MASTER_ADDRS_KEY,
+        c.get(HConstants.MASTER_ADDRS_KEY, HConstants.MASTER_ADDRS_DEFAULT));
     // Don't leave here till we've done a successful scan of the hbase:meta
     Table t = new HTable(c, TableName.META_TABLE_NAME);
     ResultScanner s = t.getScanner(new Scan());
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 2e436c6..0e715a9 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
@@ -30,6 +30,7 @@ import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -117,40 +118,88 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.log4j.AppenderSkeleton;
 import org.apache.log4j.Logger;
 import org.apache.log4j.spi.LoggingEvent;
-import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
+import org.junit.Assume;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
 /**
  * Run tests that use the HBase clients; {@link HTable}.
  * Sets up the HBase mini cluster once at start and runs through all client tests.
  * Each creates a table named for the method and does its stuff against that.
+ *
+ * Parameterized to run with different registry implementations.
  */
 @Category(LargeTests.class)
 @SuppressWarnings ("deprecation")
+@RunWith(Parameterized.class)
 public class TestFromClientSide {
   private static final Log LOG = LogFactory.getLog(TestFromClientSide.class);
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static HBaseTestingUtility TEST_UTIL;
   private static byte [] ROW = Bytes.toBytes("testRow");
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
   private static byte [] VALUE = Bytes.toBytes("testValue");
   protected static int SLAVES = 3;
 
+  @Parameterized.Parameters
+  public static Collection parameters() {
+    return Arrays.asList(new Object[][] {
+        { MasterRegistry.class },
+        { ZKConnectionRegistry.class }
+    });
+  }
+
+  // To keep the child classes happy.
+  TestFromClientSide() {}
+
+  public TestFromClientSide(Class<? extends ConnectionRegistry> registry) throws Exception {
+    initialize(registry);
+  }
+
   /**
-   * @throws java.lang.Exception
+   * JUnit does not provide an easy way to run a hook after each parameterized run. Without that
+   * there is no easy way to restart the test cluster after each parameterized run. Annotation
+   * BeforeParam does not work either because it runs before parameterization and hence does not
+   * have access to the test parameters (which is weird).
+   *
+   * This *hack* checks if the current instance of test cluster configuration has the passed
+   * parameterized configs. In such a case, we can just reuse the cluster for test and do not need
+   * to initialize from scratch. While this is a hack, it saves a ton of time for the full
+   * test and de-flakes it.
    */
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
+  protected static boolean isSameParameterizedCluster(
+      Class<? extends ConnectionRegistry> registryImpl) {
+    if (TEST_UTIL == null) {
+      return false;
+    }
+    Configuration conf = TEST_UTIL.getConfiguration();
+    Class<? extends ConnectionRegistry> confClass = conf.getClass(HConstants.REGISTRY_IMPL_CONF_KEY,
+        ZKConnectionRegistry.class, ConnectionRegistry.class);
+    return confClass.getName().equals(registryImpl.getName());
+  }
+
+  public static void initialize(Class<? extends ConnectionRegistry> registry) throws Exception {
+    // initialize() is called for every unit test, however we only want to reset the cluster state
+    // at the end of every parameterized run.
+    if (isSameParameterizedCluster(registry)) {
+      return;
+    }
+    if (TEST_UTIL != null) {
+      // We reached end of a parameterized run, clean up.
+      TEST_UTIL.shutdownMiniCluster();
+    }
+    TEST_UTIL = new HBaseTestingUtility();
     // Uncomment the following lines if more verbosity is needed for
     // debugging (see HBASE-12285 for details).
     //((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
     //((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
     //((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
     Configuration conf = TEST_UTIL.getConfiguration();
+    conf.setClass(HConstants.REGISTRY_IMPL_CONF_KEY, registry, ConnectionRegistry.class);
     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
         MultiRowMutationEndpoint.class.getName());
     conf.setBoolean("hbase.table.sanity.checks", true); // enable for below tests
@@ -168,22 +217,6 @@ public class TestFromClientSide {
   }
 
   /**
-   * @throws java.lang.Exception
-   */
-  @Before
-  public void setUp() throws Exception {
-    // Nothing to do.
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @After
-  public void tearDown() throws Exception {
-    // Nothing to do.
-  }
-
-  /**
    * Test append result when there are duplicate rpc request.
    */
   @Test
@@ -4461,6 +4494,12 @@ public class TestFromClientSide {
    */
   @Test
   public void testUnmanagedHConnectionReconnect() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    Class registryImpl = conf.getClass(
+        HConstants.REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class);
+    // This test does not make sense for MasterRegistry since it stops the only master in the
+    // cluster and starts a new master without populating the underlying config for the connection.
+    Assume.assumeFalse(registryImpl.equals(MasterRegistry.class));
     final TableName tableName = TableName.valueOf("testUnmanagedHConnectionReconnect");
     HTable t = createUnmangedHConnectionHTable(tableName);
     Connection conn = t.getConnection();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
index cd2409e..9ce0133 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
@@ -17,13 +17,18 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.util.Arrays;
+import java.util.Collection;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver;
 import org.junit.BeforeClass;
 import org.junit.experimental.categories.Category;
+import org.junit.runners.Parameterized;
 
 /**
  * Test all client operations with a coprocessor that
@@ -31,12 +36,32 @@ import org.junit.experimental.categories.Category;
  */
 @Category(LargeTests.class)
 public class TestFromClientSideWithCoprocessor extends TestFromClientSide {
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
+
+  @Parameterized.Parameters
+  public static Collection parameters() {
+    return Arrays.asList(new Object[][] {
+        { ZKConnectionRegistry.class }
+    });
+  }
+
+  public TestFromClientSideWithCoprocessor(Class registry) throws Exception {
+    initialize(registry);
+  }
+
+  public static void initialize(Class<? extends ConnectionRegistry> registry) throws Exception {
+    if (isSameParameterizedCluster(registry)) {
+      return;
+    }
+    if (TEST_UTIL != null) {
+      // We reached end of a parameterized run, clean up.
+      TEST_UTIL.shutdownMiniCluster();
+    }
+    TEST_UTIL = new HBaseTestingUtility();
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
         MultiRowMutationEndpoint.class.getName(), NoOpScanPolicyObserver.class.getName());
     conf.setBoolean("hbase.table.sanity.checks", true); // enable for below tests
+    conf.setClass(HConstants.REGISTRY_IMPL_CONF_KEY, registry, ConnectionRegistry.class);
     // We need more than one region server in this test
     TEST_UTIL.startMiniCluster(SLAVES);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
new file mode 100644
index 0000000..07f0100
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.apache.hadoop.hbase.HConstants.META_REPLICAS_NUM;
+import static org.junit.Assert.assertEquals;
+
+import com.google.common.base.Joiner;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestMasterRegistry {
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static final int META_REPLICA_COUNT = 3;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, META_REPLICA_COUNT);
+    TEST_UTIL.startMiniCluster(3, 3);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Generates a string of dummy master addresses in host:port format. Every other hostname won't
+   * have a port number.
+   */
+  private static String generateDummyMastersList(int size) {
+    List<String> masters = new ArrayList<>();
+    for (int i = 0; i < size; i++) {
+      masters.add(" localhost" + (i % 2 == 0 ? ":" + (1000 + i) : ""));
+    }
+    return Joiner.on(",").join(masters);
+  }
+
+  /**
+   * Makes sure the master registry parses the master end points in the configuration correctly.
+   */
+  @Test
+  public void testMasterAddressParsing() throws IOException {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    int numMasters = 10;
+    conf.set(HConstants.MASTER_ADDRS_KEY, generateDummyMastersList(numMasters));
+    List<ServerName> parsedMasters = new ArrayList<>(MasterRegistry.parseMasterAddrs(conf));
+    // Half of them would be without a port, duplicates are removed.
+    assertEquals(numMasters / 2 + 1, parsedMasters.size());
+    // Sort in the increasing order of port numbers.
+    Collections.sort(parsedMasters, new Comparator<ServerName>() {
+      @Override
+      public int compare(ServerName sn1, ServerName sn2) {
+        return sn1.getPort() - sn2.getPort();
+      }
+    });
+    for (int i = 0; i < parsedMasters.size(); i++) {
+      ServerName sn = parsedMasters.get(i);
+      assertEquals("localhost", sn.getHostname());
+      if (i == parsedMasters.size() - 1) {
+        // Last entry should be the one with default port.
+        assertEquals(HConstants.DEFAULT_MASTER_PORT, sn.getPort());
+      } else {
+        assertEquals(1000 + (2 * i), sn.getPort());
+      }
+    }
+  }
+
+  @Test
+  public void testRegistryRPCs() throws Exception {
+    HMaster activeMaster = TEST_UTIL.getHBaseCluster().getMaster();
+    final MasterRegistry registry = new MasterRegistry();
+    try {
+      registry.init(TEST_UTIL.getConnection());
+      // Add wait on all replicas being assigned before proceeding w/ test. Failed on occasion
+      // because not all replicas had made it up before test started.
+      TEST_UTIL.waitFor(10000, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return registry.getMetaRegionLocations().size() == META_REPLICA_COUNT;
+        }
+      });
+      assertEquals(registry.getClusterId(), activeMaster.getClusterId());
+      assertEquals(registry.getActiveMaster(), activeMaster.getServerName());
+      List<HRegionLocation> metaLocations =
+          Arrays.asList(registry.getMetaRegionLocations().getRegionLocations());
+      List<HRegionLocation> actualMetaLocations =
+          activeMaster.getMetaRegionLocationCache().getMetaRegionLocations();
+      Collections.sort(metaLocations);
+      Collections.sort(actualMetaLocations);
+      assertEquals(actualMetaLocations, metaLocations);
+      int numRs = registry.getCurrentNrHRS();
+      assertEquals(TEST_UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size(), numRs);
+    } finally {
+      registry.close();
+    }
+  }
+}