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:04 UTC

[hbase] 08/09: HBASE-24765: Dynamic master discovery (#2314)

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 3e1450d8b305856335010b305f565e0b0c823a74
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Wed Aug 26 09:13:34 2020 -0700

    HBASE-24765: Dynamic master discovery (#2314)
    
    This patch adds the ability to discover newly added masters
    dynamically on the master registry side. The trigger for the
    re-fetch is either periodic (5 mins) or any registry RPC failure.
    Master server information is cached in masters to avoid repeated
    ZK lookups.
    
    Updates the client side connection metrics to maintain a counter
    per RPC type so that clients have visibility into counts grouped
    by RPC method name.
    
    I didn't add the method to ZK registry interface since there
    is a design discussion going on in splittable meta doc. We can
    add it later if needed.
    
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
    Signed-off-by: Viraj Jasani <vj...@apache.org>
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Andrew Purtell <ap...@apache.org>
    (cherry picked from commit 275a38e1533eafa1d4bd1d50c13bcecd9a397ea8)
    (cherry picked from commit bb9121da77c7b881a3cc4c389029a610fc2b0925)
---
 .../hbase/client/MasterAddressRefresher.java       |  125 ++
 .../apache/hadoop/hbase/client/MasterRegistry.java |   67 +-
 .../hadoop/hbase/client/MetricsConnection.java     |   11 +-
 .../hbase/zookeeper/MasterAddressTracker.java      |   55 +
 .../hadoop/hbase/client/TestMetricsConnection.java |    6 +
 .../hbase/protobuf/generated/MasterProtos.java     | 1788 ++++++++++++++------
 hbase-protocol/src/main/protobuf/Master.proto      |   18 +-
 .../hadoop/hbase/master/ActiveMasterManager.java   |   42 +-
 .../org/apache/hadoop/hbase/master/HMaster.java    |   66 +-
 .../hadoop/hbase/master/MasterRpcServices.java     |   20 +-
 .../hbase/client/TestMasterAddressRefresher.java   |  176 ++
 .../hadoop/hbase/client/TestMasterRegistry.java    |  103 ++
 .../hbase/master/TestActiveMasterManager.java      |  284 ++--
 .../hbase/master/TestClientMetaServiceRPCs.java    |   21 -
 .../hadoop/hbase/master/TestMasterFailover.java    |   11 +-
 .../regionserver/TestMasterAddressTracker.java     |   37 +-
 16 files changed, 2087 insertions(+), 743 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterAddressRefresher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterAddressRefresher.java
new file mode 100644
index 0000000..08b5e9b
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterAddressRefresher.java
@@ -0,0 +1,125 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ClientMetaService;
+
+/**
+ * Thread safe utility that keeps master end points used by {@link MasterRegistry} up to date. This
+ * uses the RPC {@link ClientMetaService#getMasters} to fetch the latest list of registered masters.
+ * By default the refresh happens periodically (configured via
+ * {@link #PERIODIC_REFRESH_INTERVAL_SECS}). The refresh can also be triggered on demand via
+ * {@link #refreshNow()}. To prevent a flood of on-demand refreshes we expect that any attempts two
+ * should be spaced at least {@link #MIN_SECS_BETWEEN_REFRESHES} seconds apart.
+ */
+@InterfaceAudience.Private
+public class MasterAddressRefresher implements Closeable {
+  private static final Logger LOG = LoggerFactory.getLogger(MasterAddressRefresher.class);
+  public static final String PERIODIC_REFRESH_INTERVAL_SECS =
+      "hbase.client.master_registry.refresh_interval_secs";
+  private static final int PERIODIC_REFRESH_INTERVAL_SECS_DEFAULT = 300;
+  public static final String MIN_SECS_BETWEEN_REFRESHES =
+      "hbase.client.master_registry.min_secs_between_refreshes";
+  private static final int MIN_SECS_BETWEEN_REFRESHES_DEFAULT = 60;
+
+  private final ExecutorService pool;
+  private final MasterRegistry registry;
+  private final long periodicRefreshMs;
+  private final long timeBetweenRefreshesMs;
+  private final Object refreshMasters = new Object();
+
+  @Override
+  public void close() {
+    pool.shutdownNow();
+  }
+
+  /**
+   * Thread that refreshes the master end points until it is interrupted via {@link #close()}.
+   * Multiple callers attempting to refresh at the same time synchronize on {@link #refreshMasters}.
+   */
+  private class RefreshThread implements Runnable {
+    @Override
+    public void run() {
+      long lastRpcTs = 0;
+      while (!Thread.interrupted()) {
+        try {
+          // Spurious wake ups are okay, worst case we make an extra RPC call to refresh. We won't
+          // have duplicate refreshes because once the thread is past the wait(), notify()s are
+          // ignored until the thread is back to the waiting state.
+          synchronized (refreshMasters) {
+            refreshMasters.wait(periodicRefreshMs);
+          }
+          long currentTs = EnvironmentEdgeManager.currentTime();
+          if (lastRpcTs != 0 && currentTs - lastRpcTs <= timeBetweenRefreshesMs) {
+            continue;
+          }
+          lastRpcTs = currentTs;
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(registry.getMasters());
+          registry.populateMasterStubs(newMasters);
+          LOG.debug("Finished refreshing master end points. {}", newMasters);
+        } catch (InterruptedException e) {
+          LOG.debug("Interrupted during wait, aborting refresh-masters-thread.", e);
+          break;
+        } catch (IOException e) {
+          LOG.debug("Error populating latest list of masters.", e);
+        }
+      }
+      LOG.info("Master end point refresher loop exited.");
+    }
+  }
+
+  MasterAddressRefresher(Configuration conf, MasterRegistry registry) {
+    pool = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("master-registry-refresh-end-points").setDaemon(true).build());
+    periodicRefreshMs = TimeUnit.SECONDS.toMillis(conf.getLong(PERIODIC_REFRESH_INTERVAL_SECS,
+        PERIODIC_REFRESH_INTERVAL_SECS_DEFAULT));
+    timeBetweenRefreshesMs = TimeUnit.SECONDS.toMillis(conf.getLong(MIN_SECS_BETWEEN_REFRESHES,
+        MIN_SECS_BETWEEN_REFRESHES_DEFAULT));
+    Preconditions.checkArgument(periodicRefreshMs > 0);
+    Preconditions.checkArgument(timeBetweenRefreshesMs < periodicRefreshMs);
+    this.registry = registry;
+    pool.submit(new RefreshThread());
+  }
+
+  /**
+   * Notifies the refresher thread to refresh the configuration. This does not guarantee a refresh.
+   * See class comment for details.
+   */
+  void refreshNow() {
+    synchronized (refreshMasters) {
+      refreshMasters.notify();
+    }
+  }
+}
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
index 882173f..877049c 100644
--- 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
@@ -22,6 +22,7 @@ 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.collect.ImmutableSet;
 import com.google.common.net.HostAndPort;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcController;
@@ -40,6 +41,7 @@ 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.ClientExceptionsUtil;
 import org.apache.hadoop.hbase.exceptions.MasterRegistryFetchException;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
@@ -51,10 +53,11 @@ 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.GetMastersRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry;
 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;
@@ -69,13 +72,15 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNumLiveRSRespo
 public class MasterRegistry implements ConnectionRegistry {
   private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
 
-  private ImmutableMap<String, ClientMetaService.Interface> masterAddr2Stub;
+  private volatile ImmutableMap<String, ClientMetaService.Interface> masterAddr2Stub;
 
   // RPC client used to talk to the masters.
   private RpcClient rpcClient;
   private RpcControllerFactory rpcControllerFactory;
   private int rpcTimeoutMs;
 
+  protected MasterAddressRefresher masterAddressRefresher;
+
   @Override
   public void init(Connection connection) throws IOException {
     Configuration conf = connection.getConfiguration();
@@ -87,13 +92,15 @@ public class MasterRegistry implements ConnectionRegistry {
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
     populateMasterStubs(parseMasterAddrs(conf));
+    masterAddressRefresher = new MasterAddressRefresher(conf, this);
   }
 
-  private interface Callable <T extends Message> {
+  protected 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 {
+   protected <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());
@@ -102,14 +109,16 @@ public class MasterRegistry implements ConnectionRegistry {
       HBaseRpcController controller = rpcControllerFactory.newController();
       try {
         T resp = callable.call(stub, controller);
-        if (controller.failed()) {
-          lastException = controller.getFailed();
-          continue;
+        if (!controller.failed()) {
+          return resp;
         }
-        return resp;
+        lastException = controller.getFailed();
       } catch (Exception e) {
         lastException = e;
       }
+      if (ClientExceptionsUtil.isConnectionException(lastException)) {
+        masterAddressRefresher.refreshNow();
+      }
     }
     // rpcs to all masters failed.
     throw new MasterRegistryFetchException(masters, lastException);
@@ -117,19 +126,37 @@ public class MasterRegistry implements ConnectionRegistry {
 
   @Override
   public ServerName getActiveMaster() throws IOException {
-    GetActiveMasterResponse resp = doCall(new Callable<GetActiveMasterResponse>() {
+    GetMastersResponseEntry activeMaster = null;
+    for (GetMastersResponseEntry entry: getMastersInternal().getMasterServersList()) {
+      if (entry.getIsActive()) {
+        activeMaster = entry;
+        break;
+      }
+    }
+    if (activeMaster == null) {
+      throw new HBaseIOException("No active master found");
+    }
+    return ProtobufUtil.toServerName(activeMaster.getServerName());
+  }
+
+  List<ServerName> getMasters() throws IOException {
+    List<ServerName> result = new ArrayList<>();
+    for (GetMastersResponseEntry entry: getMastersInternal().getMasterServersList()) {
+      result.add(ProtobufUtil.toServerName(entry.getServerName()));
+    }
+    return result;
+  }
+
+  private GetMastersResponse getMastersInternal() throws IOException {
+    return doCall(new Callable<GetMastersResponse>() {
       @Override
-      public GetActiveMasterResponse call(
+      public GetMastersResponse call(
           ClientMetaService.Interface stub, RpcController controller) throws IOException {
-        BlockingRpcCallback<GetActiveMasterResponse> cb = new BlockingRpcCallback<>();
-        stub.getActiveMaster(controller, GetActiveMasterRequest.getDefaultInstance(), cb);
+        BlockingRpcCallback<GetMastersResponse> cb = new BlockingRpcCallback<>();
+        stub.getMasters(controller, GetMastersRequest.getDefaultInstance(), cb);
         return cb.get();
       }
     });
-    if (!resp.hasServerName() || resp.getServerName() == null) {
-      throw new HBaseIOException("No active master found");
-    }
-    return ProtobufUtil.toServerName(resp.getServerName());
   }
 
   @Override
@@ -230,4 +257,10 @@ public class MasterRegistry implements ConnectionRegistry {
     }
     masterAddr2Stub = builder.build();
   }
+
+  @InterfaceAudience.Private
+  ImmutableSet<String> getParsedMasterServers() {
+    return masterAddr2Stub.keySet();
+  }
+
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
index 6328d7f..0bce8eb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
@@ -56,6 +56,7 @@ public class MetricsConnection implements StatisticTrackable {
   /** Set this key to {@code true} to enable metrics collection of client requests. */
   public static final String CLIENT_SIDE_METRICS_ENABLED_KEY = "hbase.client.metrics.enable";
 
+  private static final String CNT_BASE = "rpcCount_";
   private static final String DRTN_BASE = "rpcCallDurationMs_";
   private static final String REQ_BASE = "rpcCallRequestSizeBytes_";
   private static final String RESP_BASE = "rpcCallResponseSizeBytes_";
@@ -303,6 +304,8 @@ public class MetricsConnection implements StatisticTrackable {
           LOAD_FACTOR, CONCURRENCY_LEVEL);
   private final ConcurrentMap<String, Counter> cacheDroppingExceptions =
     new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
+  @VisibleForTesting protected final ConcurrentMap<String, Counter>  rpcCounters =
+      new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
 
   public MetricsConnection(final ConnectionManager.HConnectionImplementation conn) {
     this.scope = conn.toString();
@@ -450,8 +453,7 @@ public class MetricsConnection implements StatisticTrackable {
   }
 
   /** Update call stats for non-critical-path methods */
-  private void updateRpcGeneric(MethodDescriptor method, CallStats stats) {
-    final String methodName = method.getService().getName() + "_" + method.getName();
+  private void updateRpcGeneric(String methodName, CallStats stats) {
     getMetric(DRTN_BASE + methodName, rpcTimers, timerFactory)
         .update(stats.getCallTimeMs(), TimeUnit.MILLISECONDS);
     getMetric(REQ_BASE + methodName, rpcHistograms, histogramFactory)
@@ -466,6 +468,9 @@ public class MetricsConnection implements StatisticTrackable {
     if (callsPerServer > 0) {
       concurrentCallsPerServerHist.update(callsPerServer);
     }
+    // Update the counter that tracks RPCs by type.
+    final String methodName = method.getService().getName() + "_" + method.getName();
+    getMetric(CNT_BASE + methodName, rpcCounters, counterFactory).inc();
     // this implementation is tied directly to protobuf implementation details. would be better
     // if we could dispatch based on something static, ie, request Message type.
     if (method.getService() == ClientService.getDescriptor()) {
@@ -518,7 +523,7 @@ public class MetricsConnection implements StatisticTrackable {
       }
     }
     // Fallback to dynamic registry lookup for DDL methods.
-    updateRpcGeneric(method, stats);
+    updateRpcGeneric(methodName, stats);
   }
 
   public void incrCacheDroppingExceptions(Object exception) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
index 311202c..c34d294 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hbase.zookeeper;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HConstants;
@@ -68,6 +72,57 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker {
   }
 
   /**
+   * @param watcher ZooKeeperWatcher instance to use for querying ZK.
+   * @return current list of backup masters.
+   */
+  public static List<ServerName> getBackupMastersAndRenewWatch(
+      ZooKeeperWatcher watcher) {
+    // Build Set of backup masters from ZK nodes
+    List<String> backupMasterStrings;
+    try {
+      backupMasterStrings = ZKUtil.listChildrenAndWatchForNewChildren(
+          watcher, watcher.backupMasterAddressesZNode);
+    } catch (KeeperException e) {
+      LOG.warn(watcher.prefix("Unable to list backup servers"), e);
+      backupMasterStrings = null;
+    }
+
+    List<ServerName> backupMasters = new ArrayList<>();
+    if (backupMasterStrings != null && !backupMasterStrings.isEmpty()) {
+      for (String s: backupMasterStrings) {
+        try {
+          byte [] bytes;
+          try {
+            bytes = ZKUtil.getData(watcher, ZKUtil.joinZNode(
+               watcher.backupMasterAddressesZNode, s));
+          } catch (InterruptedException e) {
+            throw new InterruptedIOException("Thread interrupted.");
+          }
+          if (bytes != null) {
+            ServerName sn;
+            try {
+              sn = ServerName.parseFrom(bytes);
+            } catch (DeserializationException e) {
+              LOG.warn("Failed parse, skipping registering backup server", e);
+              continue;
+            }
+            backupMasters.add(sn);
+          }
+        } catch (KeeperException | InterruptedIOException e) {
+          LOG.warn(watcher.prefix("Unable to get information about " +
+              "backup servers"), e);
+        }
+      }
+      Collections.sort(backupMasters, new Comparator<ServerName>() {
+        @Override
+        public int compare(ServerName s1, ServerName s2) {
+          return s1.getServerName().compareTo(s2.getServerName());
+        }});
+    }
+    return backupMasters;
+  }
+
+  /**
    * Get the address of the current master if one is available.  Returns null
    * if no current master.
    * @return Server name or null if timed out.
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java
index f1be81b..9638c72 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import com.google.protobuf.ByteString;
 import com.yammer.metrics.util.RatioGauge;
@@ -129,6 +130,11 @@ public class TestMetricsConnection {
               .build(),
           MetricsConnection.newCallStats());
     }
+    for (String method: new String[]{"Get", "Scan", "Mutate"}) {
+      final String metricKey = "rpcCount_" + ClientService.getDescriptor().getName() + "_" + method;
+      final long metricVal = METRICS.rpcCounters.get(metricKey).count();
+      assertTrue("metric: " + metricKey + " val: " + metricVal, metricVal >= loop);
+    }
     for (MetricsConnection.CallTracker t : new MetricsConnection.CallTracker[] {
       METRICS.getTracker, METRICS.scanTracker, METRICS.multiTracker, METRICS.appendTracker,
       METRICS.deleteTracker, METRICS.incrementTracker, METRICS.putTracker
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 909c144..2eaed11 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
@@ -66722,32 +66722,32 @@ public final class MasterProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.GetClusterIdResponse)
   }
 
-  public interface GetActiveMasterRequestOrBuilder
+  public interface GetMastersRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
   }
   /**
-   * Protobuf type {@code hbase.pb.GetActiveMasterRequest}
+   * Protobuf type {@code hbase.pb.GetMastersRequest}
    *
    * <pre>
-   ** Request and response to get the currently active master name for this cluster 
+   ** Request and response to get the current list of all registers master servers 
    * </pre>
    */
-  public static final class GetActiveMasterRequest extends
+  public static final class GetMastersRequest extends
       com.google.protobuf.GeneratedMessage
-      implements GetActiveMasterRequestOrBuilder {
-    // Use GetActiveMasterRequest.newBuilder() to construct.
-    private GetActiveMasterRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements GetMastersRequestOrBuilder {
+    // Use GetMastersRequest.newBuilder() to construct.
+    private GetMastersRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private GetActiveMasterRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private GetMastersRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final GetActiveMasterRequest defaultInstance;
-    public static GetActiveMasterRequest getDefaultInstance() {
+    private static final GetMastersRequest defaultInstance;
+    public static GetMastersRequest getDefaultInstance() {
       return defaultInstance;
     }
 
-    public GetActiveMasterRequest getDefaultInstanceForType() {
+    public GetMastersRequest getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -66757,7 +66757,7 @@ public final class MasterProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private GetActiveMasterRequest(
+    private GetMastersRequest(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -66793,28 +66793,28 @@ public final class MasterProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetActiveMasterRequest_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersRequest_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetActiveMasterRequest_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<GetActiveMasterRequest> PARSER =
-        new com.google.protobuf.AbstractParser<GetActiveMasterRequest>() {
-      public GetActiveMasterRequest parsePartialFrom(
+    public static com.google.protobuf.Parser<GetMastersRequest> PARSER =
+        new com.google.protobuf.AbstractParser<GetMastersRequest>() {
+      public GetMastersRequest parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new GetActiveMasterRequest(input, extensionRegistry);
+        return new GetMastersRequest(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<GetActiveMasterRequest> getParserForType() {
+    public com.google.protobuf.Parser<GetMastersRequest> getParserForType() {
       return PARSER;
     }
 
@@ -66858,10 +66858,10 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest) obj;
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest) obj;
 
       boolean result = true;
       result = result &&
@@ -66882,53 +66882,53 @@ public final class MasterProtos {
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest 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.GetActiveMasterRequest parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest 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.GetActiveMasterRequest parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest 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.GetActiveMasterRequest parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest 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.GetActiveMasterRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -66937,7 +66937,7 @@ public final class MasterProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -66949,28 +66949,28 @@ public final class MasterProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.GetActiveMasterRequest}
+     * Protobuf type {@code hbase.pb.GetMastersRequest}
      *
      * <pre>
-     ** Request and response to get the currently active master name for this cluster 
+     ** Request and response to get the current list of all registers master servers 
      * </pre>
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequestOrBuilder {
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequestOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetActiveMasterRequest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersRequest_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetActiveMasterRequest_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest.Builder.class);
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -66999,38 +66999,38 @@ public final class MasterProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetActiveMasterRequest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersRequest_descriptor;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest.getDefaultInstance();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest result = buildPartial();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest(this);
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest(this);
         onBuilt();
         return result;
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest)other);
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest.getDefaultInstance()) return this;
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest.getDefaultInstance()) return this;
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -67043,11 +67043,11 @@ public final class MasterProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest parsedMessage = null;
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -67057,65 +67057,723 @@ public final class MasterProtos {
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.GetActiveMasterRequest)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.GetMastersRequest)
     }
 
     static {
-      defaultInstance = new GetActiveMasterRequest(true);
+      defaultInstance = new GetMastersRequest(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.GetActiveMasterRequest)
+    // @@protoc_insertion_point(class_scope:hbase.pb.GetMastersRequest)
   }
 
-  public interface GetActiveMasterResponseOrBuilder
+  public interface GetMastersResponseEntryOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // optional .hbase.pb.ServerName server_name = 1;
+    // required .hbase.pb.ServerName server_name = 1;
     /**
-     * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-     *
-     * <pre>
-     ** Not set if an active master could not be determined. 
-     * </pre>
+     * <code>required .hbase.pb.ServerName server_name = 1;</code>
      */
     boolean hasServerName();
     /**
-     * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-     *
-     * <pre>
-     ** Not set if an active master could not be determined. 
-     * </pre>
+     * <code>required .hbase.pb.ServerName server_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName();
+    /**
+     * <code>required .hbase.pb.ServerName server_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder();
+
+    // required bool is_active = 2;
+    /**
+     * <code>required bool is_active = 2;</code>
+     */
+    boolean hasIsActive();
+    /**
+     * <code>required bool is_active = 2;</code>
+     */
+    boolean getIsActive();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.GetMastersResponseEntry}
+   */
+  public static final class GetMastersResponseEntry extends
+      com.google.protobuf.GeneratedMessage
+      implements GetMastersResponseEntryOrBuilder {
+    // Use GetMastersResponseEntry.newBuilder() to construct.
+    private GetMastersResponseEntry(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private GetMastersResponseEntry(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final GetMastersResponseEntry defaultInstance;
+    public static GetMastersResponseEntry getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public GetMastersResponseEntry getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GetMastersResponseEntry(
+        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 10: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = serverName_.toBuilder();
+              }
+              serverName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(serverName_);
+                serverName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              isActive_ = input.readBool();
+              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_GetMastersResponseEntry_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersResponseEntry_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<GetMastersResponseEntry> PARSER =
+        new com.google.protobuf.AbstractParser<GetMastersResponseEntry>() {
+      public GetMastersResponseEntry parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new GetMastersResponseEntry(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<GetMastersResponseEntry> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required .hbase.pb.ServerName server_name = 1;
+    public static final int SERVER_NAME_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName serverName_;
+    /**
+     * <code>required .hbase.pb.ServerName server_name = 1;</code>
+     */
+    public boolean hasServerName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .hbase.pb.ServerName server_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName() {
+      return serverName_;
+    }
+    /**
+     * <code>required .hbase.pb.ServerName server_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder() {
+      return serverName_;
+    }
+
+    // required bool is_active = 2;
+    public static final int IS_ACTIVE_FIELD_NUMBER = 2;
+    private boolean isActive_;
+    /**
+     * <code>required bool is_active = 2;</code>
+     */
+    public boolean hasIsActive() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required bool is_active = 2;</code>
+     */
+    public boolean getIsActive() {
+      return isActive_;
+    }
+
+    private void initFields() {
+      serverName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+      isActive_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasServerName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasIsActive()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getServerName().isInitialized()) {
+        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.writeMessage(1, serverName_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBool(2, isActive_);
+      }
+      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
+          .computeMessageSize(1, serverName_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(2, isActive_);
+      }
+      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.GetMastersResponseEntry)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry) obj;
+
+      boolean result = true;
+      result = result && (hasServerName() == other.hasServerName());
+      if (hasServerName()) {
+        result = result && getServerName()
+            .equals(other.getServerName());
+      }
+      result = result && (hasIsActive() == other.hasIsActive());
+      if (hasIsActive()) {
+        result = result && (getIsActive()
+            == other.getIsActive());
+      }
+      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 (hasServerName()) {
+        hash = (37 * hash) + SERVER_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getServerName().hashCode();
+      }
+      if (hasIsActive()) {
+        hash = (37 * hash) + IS_ACTIVE_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getIsActive());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry 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.GetMastersResponseEntry parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry 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.GetMastersResponseEntry parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry 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.GetMastersResponseEntry parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry 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.GetMastersResponseEntry parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry 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.GetMastersResponseEntry 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.GetMastersResponseEntry}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntryOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersResponseEntry_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersResponseEntry_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getServerNameFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (serverNameBuilder_ == null) {
+          serverName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+        } else {
+          serverNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        isActive_ = false;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        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_GetMastersResponseEntry_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (serverNameBuilder_ == null) {
+          result.serverName_ = serverName_;
+        } else {
+          result.serverName_ = serverNameBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.isActive_ = isActive_;
+        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.GetMastersResponseEntry) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.getDefaultInstance()) return this;
+        if (other.hasServerName()) {
+          mergeServerName(other.getServerName());
+        }
+        if (other.hasIsActive()) {
+          setIsActive(other.getIsActive());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasServerName()) {
+          
+          return false;
+        }
+        if (!hasIsActive()) {
+          
+          return false;
+        }
+        if (!getServerName().isInitialized()) {
+          
+          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.GetMastersResponseEntry parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required .hbase.pb.ServerName server_name = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName serverName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverNameBuilder_;
+      /**
+       * <code>required .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public boolean hasServerName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName() {
+        if (serverNameBuilder_ == null) {
+          return serverName_;
+        } else {
+          return serverNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public Builder setServerName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) {
+        if (serverNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          serverName_ = value;
+          onChanged();
+        } else {
+          serverNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public Builder setServerName(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) {
+        if (serverNameBuilder_ == null) {
+          serverName_ = builderForValue.build();
+          onChanged();
+        } else {
+          serverNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public Builder mergeServerName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) {
+        if (serverNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              serverName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) {
+            serverName_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(serverName_).mergeFrom(value).buildPartial();
+          } else {
+            serverName_ = value;
+          }
+          onChanged();
+        } else {
+          serverNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public Builder clearServerName() {
+        if (serverNameBuilder_ == null) {
+          serverName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+          onChanged();
+        } else {
+          serverNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder getServerNameBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getServerNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder() {
+        if (serverNameBuilder_ != null) {
+          return serverNameBuilder_.getMessageOrBuilder();
+        } else {
+          return serverName_;
+        }
+      }
+      /**
+       * <code>required .hbase.pb.ServerName server_name = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
+          getServerNameFieldBuilder() {
+        if (serverNameBuilder_ == null) {
+          serverNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
+                  serverName_,
+                  getParentForChildren(),
+                  isClean());
+          serverName_ = null;
+        }
+        return serverNameBuilder_;
+      }
+
+      // required bool is_active = 2;
+      private boolean isActive_ ;
+      /**
+       * <code>required bool is_active = 2;</code>
+       */
+      public boolean hasIsActive() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required bool is_active = 2;</code>
+       */
+      public boolean getIsActive() {
+        return isActive_;
+      }
+      /**
+       * <code>required bool is_active = 2;</code>
+       */
+      public Builder setIsActive(boolean value) {
+        bitField0_ |= 0x00000002;
+        isActive_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bool is_active = 2;</code>
+       */
+      public Builder clearIsActive() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        isActive_ = false;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.GetMastersResponseEntry)
+    }
+
+    static {
+      defaultInstance = new GetMastersResponseEntry(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.GetMastersResponseEntry)
+  }
+
+  public interface GetMastersResponseOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;
+    /**
+     * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry> 
+        getMasterServersList();
+    /**
+     * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry getMasterServers(int index);
+    /**
+     * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+     */
+    int getMasterServersCount();
+    /**
+     * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName();
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntryOrBuilder> 
+        getMasterServersOrBuilderList();
     /**
-     * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-     *
-     * <pre>
-     ** Not set if an active master could not be determined. 
-     * </pre>
+     * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder();
+    org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntryOrBuilder getMasterServersOrBuilder(
+        int index);
   }
   /**
-   * Protobuf type {@code hbase.pb.GetActiveMasterResponse}
+   * Protobuf type {@code hbase.pb.GetMastersResponse}
    */
-  public static final class GetActiveMasterResponse extends
+  public static final class GetMastersResponse extends
       com.google.protobuf.GeneratedMessage
-      implements GetActiveMasterResponseOrBuilder {
-    // Use GetActiveMasterResponse.newBuilder() to construct.
-    private GetActiveMasterResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements GetMastersResponseOrBuilder {
+    // Use GetMastersResponse.newBuilder() to construct.
+    private GetMastersResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private GetActiveMasterResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private GetMastersResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final GetActiveMasterResponse defaultInstance;
-    public static GetActiveMasterResponse getDefaultInstance() {
+    private static final GetMastersResponse defaultInstance;
+    public static GetMastersResponse getDefaultInstance() {
       return defaultInstance;
     }
 
-    public GetActiveMasterResponse getDefaultInstanceForType() {
+    public GetMastersResponse getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -67125,7 +67783,7 @@ public final class MasterProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private GetActiveMasterResponse(
+    private GetMastersResponse(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -67149,16 +67807,11 @@ public final class MasterProtos {
               break;
             }
             case 10: {
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                subBuilder = serverName_.toBuilder();
-              }
-              serverName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(serverName_);
-                serverName_ = subBuilder.buildPartial();
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                masterServers_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry>();
+                mutable_bitField0_ |= 0x00000001;
               }
-              bitField0_ |= 0x00000001;
+              masterServers_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.PARSER, extensionRegistry));
               break;
             }
           }
@@ -67169,82 +67822,86 @@ public final class MasterProtos {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          masterServers_ = java.util.Collections.unmodifiableList(masterServers_);
+        }
         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_GetActiveMasterResponse_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersResponse_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetActiveMasterResponse_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<GetActiveMasterResponse> PARSER =
-        new com.google.protobuf.AbstractParser<GetActiveMasterResponse>() {
-      public GetActiveMasterResponse parsePartialFrom(
+    public static com.google.protobuf.Parser<GetMastersResponse> PARSER =
+        new com.google.protobuf.AbstractParser<GetMastersResponse>() {
+      public GetMastersResponse parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new GetActiveMasterResponse(input, extensionRegistry);
+        return new GetMastersResponse(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<GetActiveMasterResponse> getParserForType() {
+    public com.google.protobuf.Parser<GetMastersResponse> getParserForType() {
       return PARSER;
     }
 
-    private int bitField0_;
-    // optional .hbase.pb.ServerName server_name = 1;
-    public static final int SERVER_NAME_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName serverName_;
+    // repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;
+    public static final int MASTER_SERVERS_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry> masterServers_;
     /**
-     * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-     *
-     * <pre>
-     ** Not set if an active master could not be determined. 
-     * </pre>
+     * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
      */
-    public boolean hasServerName() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry> getMasterServersList() {
+      return masterServers_;
     }
     /**
-     * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-     *
-     * <pre>
-     ** Not set if an active master could not be determined. 
-     * </pre>
+     * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName() {
-      return serverName_;
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntryOrBuilder> 
+        getMasterServersOrBuilderList() {
+      return masterServers_;
     }
     /**
-     * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-     *
-     * <pre>
-     ** Not set if an active master could not be determined. 
-     * </pre>
+     * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder() {
-      return serverName_;
+    public int getMasterServersCount() {
+      return masterServers_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry getMasterServers(int index) {
+      return masterServers_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntryOrBuilder getMasterServersOrBuilder(
+        int index) {
+      return masterServers_.get(index);
     }
 
     private void initFields() {
-      serverName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+      masterServers_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
-      if (hasServerName()) {
-        if (!getServerName().isInitialized()) {
+      for (int i = 0; i < getMasterServersCount(); i++) {
+        if (!getMasterServers(i).isInitialized()) {
           memoizedIsInitialized = 0;
           return false;
         }
@@ -67256,8 +67913,8 @@ public final class MasterProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeMessage(1, serverName_);
+      for (int i = 0; i < masterServers_.size(); i++) {
+        output.writeMessage(1, masterServers_.get(i));
       }
       getUnknownFields().writeTo(output);
     }
@@ -67268,9 +67925,9 @@ public final class MasterProtos {
       if (size != -1) return size;
 
       size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+      for (int i = 0; i < masterServers_.size(); i++) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(1, serverName_);
+          .computeMessageSize(1, masterServers_.get(i));
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -67289,17 +67946,14 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse) obj;
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse) obj;
 
       boolean result = true;
-      result = result && (hasServerName() == other.hasServerName());
-      if (hasServerName()) {
-        result = result && getServerName()
-            .equals(other.getServerName());
-      }
+      result = result && getMasterServersList()
+          .equals(other.getMasterServersList());
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -67313,62 +67967,62 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasServerName()) {
-        hash = (37 * hash) + SERVER_NAME_FIELD_NUMBER;
-        hash = (53 * hash) + getServerName().hashCode();
+      if (getMasterServersCount() > 0) {
+        hash = (37 * hash) + MASTER_SERVERS_FIELD_NUMBER;
+        hash = (53 * hash) + getMasterServersList().hashCode();
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse 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.GetActiveMasterResponse parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse 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.GetActiveMasterResponse parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse 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.GetActiveMasterResponse parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse 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.GetActiveMasterResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -67377,7 +68031,7 @@ public final class MasterProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -67389,24 +68043,24 @@ public final class MasterProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.GetActiveMasterResponse}
+     * Protobuf type {@code hbase.pb.GetMastersResponse}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponseOrBuilder {
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetActiveMasterResponse_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersResponse_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetActiveMasterResponse_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.Builder.class);
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -67418,7 +68072,7 @@ public final class MasterProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getServerNameFieldBuilder();
+          getMasterServersFieldBuilder();
         }
       }
       private static Builder create() {
@@ -67427,12 +68081,12 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
-        if (serverNameBuilder_ == null) {
-          serverName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+        if (masterServersBuilder_ == null) {
+          masterServers_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
         } else {
-          serverNameBuilder_.clear();
+          masterServersBuilder_.clear();
         }
-        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
@@ -67442,59 +68096,81 @@ public final class MasterProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetActiveMasterResponse_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetMastersResponse_descriptor;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.getDefaultInstance();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse result = buildPartial();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse(this);
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse(this);
         int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        if (serverNameBuilder_ == null) {
-          result.serverName_ = serverName_;
+        if (masterServersBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            masterServers_ = java.util.Collections.unmodifiableList(masterServers_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.masterServers_ = masterServers_;
         } else {
-          result.serverName_ = serverNameBuilder_.build();
+          result.masterServers_ = masterServersBuilder_.build();
         }
-        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.GetActiveMasterResponse) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse)other);
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.getDefaultInstance()) return this;
-        if (other.hasServerName()) {
-          mergeServerName(other.getServerName());
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.getDefaultInstance()) return this;
+        if (masterServersBuilder_ == null) {
+          if (!other.masterServers_.isEmpty()) {
+            if (masterServers_.isEmpty()) {
+              masterServers_ = other.masterServers_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensureMasterServersIsMutable();
+              masterServers_.addAll(other.masterServers_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.masterServers_.isEmpty()) {
+            if (masterServersBuilder_.isEmpty()) {
+              masterServersBuilder_.dispose();
+              masterServersBuilder_ = null;
+              masterServers_ = other.masterServers_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              masterServersBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getMasterServersFieldBuilder() : null;
+            } else {
+              masterServersBuilder_.addAllMessages(other.masterServers_);
+            }
+          }
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
 
       public final boolean isInitialized() {
-        if (hasServerName()) {
-          if (!getServerName().isInitialized()) {
+        for (int i = 0; i < getMasterServersCount(); i++) {
+          if (!getMasterServers(i).isInitialized()) {
             
             return false;
           }
@@ -67506,11 +68182,11 @@ public final class MasterProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse parsedMessage = null;
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -67521,168 +68197,255 @@ public final class MasterProtos {
       }
       private int bitField0_;
 
-      // optional .hbase.pb.ServerName server_name = 1;
-      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName serverName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverNameBuilder_;
+      // repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry> masterServers_ =
+        java.util.Collections.emptyList();
+      private void ensureMasterServersIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          masterServers_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry>(masterServers_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.Builder, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntryOrBuilder> masterServersBuilder_;
+
       /**
-       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-       *
-       * <pre>
-       ** Not set if an active master could not be determined. 
-       * </pre>
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
        */
-      public boolean hasServerName() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry> getMasterServersList() {
+        if (masterServersBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(masterServers_);
+        } else {
+          return masterServersBuilder_.getMessageList();
+        }
       }
       /**
-       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-       *
-       * <pre>
-       ** Not set if an active master could not be determined. 
-       * </pre>
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName() {
-        if (serverNameBuilder_ == null) {
-          return serverName_;
+      public int getMasterServersCount() {
+        if (masterServersBuilder_ == null) {
+          return masterServers_.size();
         } else {
-          return serverNameBuilder_.getMessage();
+          return masterServersBuilder_.getCount();
         }
       }
       /**
-       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-       *
-       * <pre>
-       ** Not set if an active master could not be determined. 
-       * </pre>
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
        */
-      public Builder setServerName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) {
-        if (serverNameBuilder_ == null) {
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry getMasterServers(int index) {
+        if (masterServersBuilder_ == null) {
+          return masterServers_.get(index);
+        } else {
+          return masterServersBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+       */
+      public Builder setMasterServers(
+          int index, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry value) {
+        if (masterServersBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          serverName_ = value;
+          ensureMasterServersIsMutable();
+          masterServers_.set(index, value);
           onChanged();
         } else {
-          serverNameBuilder_.setMessage(value);
+          masterServersBuilder_.setMessage(index, value);
         }
-        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-       *
-       * <pre>
-       ** Not set if an active master could not be determined. 
-       * </pre>
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
        */
-      public Builder setServerName(
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) {
-        if (serverNameBuilder_ == null) {
-          serverName_ = builderForValue.build();
+      public Builder setMasterServers(
+          int index, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.Builder builderForValue) {
+        if (masterServersBuilder_ == null) {
+          ensureMasterServersIsMutable();
+          masterServers_.set(index, builderForValue.build());
           onChanged();
         } else {
-          serverNameBuilder_.setMessage(builderForValue.build());
+          masterServersBuilder_.setMessage(index, builderForValue.build());
         }
-        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-       *
-       * <pre>
-       ** Not set if an active master could not be determined. 
-       * </pre>
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
        */
-      public Builder mergeServerName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) {
-        if (serverNameBuilder_ == null) {
-          if (((bitField0_ & 0x00000001) == 0x00000001) &&
-              serverName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) {
-            serverName_ =
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(serverName_).mergeFrom(value).buildPartial();
-          } else {
-            serverName_ = value;
+      public Builder addMasterServers(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry value) {
+        if (masterServersBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
           }
+          ensureMasterServersIsMutable();
+          masterServers_.add(value);
           onChanged();
         } else {
-          serverNameBuilder_.mergeFrom(value);
+          masterServersBuilder_.addMessage(value);
         }
-        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-       *
-       * <pre>
-       ** Not set if an active master could not be determined. 
-       * </pre>
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
        */
-      public Builder clearServerName() {
-        if (serverNameBuilder_ == null) {
-          serverName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+      public Builder addMasterServers(
+          int index, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry value) {
+        if (masterServersBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureMasterServersIsMutable();
+          masterServers_.add(index, value);
           onChanged();
         } else {
-          serverNameBuilder_.clear();
+          masterServersBuilder_.addMessage(index, value);
         }
-        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
       /**
-       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-       *
-       * <pre>
-       ** Not set if an active master could not be determined. 
-       * </pre>
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder getServerNameBuilder() {
-        bitField0_ |= 0x00000001;
-        onChanged();
-        return getServerNameFieldBuilder().getBuilder();
+      public Builder addMasterServers(
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.Builder builderForValue) {
+        if (masterServersBuilder_ == null) {
+          ensureMasterServersIsMutable();
+          masterServers_.add(builderForValue.build());
+          onChanged();
+        } else {
+          masterServersBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
       }
       /**
-       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-       *
-       * <pre>
-       ** Not set if an active master could not be determined. 
-       * </pre>
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder() {
-        if (serverNameBuilder_ != null) {
-          return serverNameBuilder_.getMessageOrBuilder();
+      public Builder addMasterServers(
+          int index, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.Builder builderForValue) {
+        if (masterServersBuilder_ == null) {
+          ensureMasterServersIsMutable();
+          masterServers_.add(index, builderForValue.build());
+          onChanged();
         } else {
-          return serverName_;
+          masterServersBuilder_.addMessage(index, builderForValue.build());
         }
+        return this;
       }
       /**
-       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
-       *
-       * <pre>
-       ** Not set if an active master could not be determined. 
-       * </pre>
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
        */
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
-          getServerNameFieldBuilder() {
-        if (serverNameBuilder_ == null) {
-          serverNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
-                  serverName_,
+      public Builder addAllMasterServers(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry> values) {
+        if (masterServersBuilder_ == null) {
+          ensureMasterServersIsMutable();
+          super.addAll(values, masterServers_);
+          onChanged();
+        } else {
+          masterServersBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+       */
+      public Builder clearMasterServers() {
+        if (masterServersBuilder_ == null) {
+          masterServers_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          masterServersBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+       */
+      public Builder removeMasterServers(int index) {
+        if (masterServersBuilder_ == null) {
+          ensureMasterServersIsMutable();
+          masterServers_.remove(index);
+          onChanged();
+        } else {
+          masterServersBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.Builder getMasterServersBuilder(
+          int index) {
+        return getMasterServersFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntryOrBuilder getMasterServersOrBuilder(
+          int index) {
+        if (masterServersBuilder_ == null) {
+          return masterServers_.get(index);  } else {
+          return masterServersBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntryOrBuilder> 
+           getMasterServersOrBuilderList() {
+        if (masterServersBuilder_ != null) {
+          return masterServersBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(masterServers_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.Builder addMasterServersBuilder() {
+        return getMasterServersFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.Builder addMasterServersBuilder(
+          int index) {
+        return getMasterServersFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.GetMastersResponseEntry master_servers = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.Builder> 
+           getMasterServersBuilderList() {
+        return getMasterServersFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.Builder, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntryOrBuilder> 
+          getMasterServersFieldBuilder() {
+        if (masterServersBuilder_ == null) {
+          masterServersBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry.Builder, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntryOrBuilder>(
+                  masterServers_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
                   getParentForChildren(),
                   isClean());
-          serverName_ = null;
+          masterServers_ = null;
         }
-        return serverNameBuilder_;
+        return masterServersBuilder_;
       }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.GetActiveMasterResponse)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.GetMastersResponse)
     }
 
     static {
-      defaultInstance = new GetActiveMasterResponse(true);
+      defaultInstance = new GetMastersResponse(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.GetActiveMasterResponse)
+    // @@protoc_insertion_point(class_scope:hbase.pb.GetMastersResponse)
   }
 
   public interface GetMetaRegionLocationsRequestOrBuilder
@@ -75043,17 +75806,18 @@ public final class MasterProtos {
           com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdResponse> done);
 
       /**
-       * <code>rpc GetActiveMaster(.hbase.pb.GetActiveMasterRequest) returns (.hbase.pb.GetActiveMasterResponse);</code>
+       * <code>rpc GetMasters(.hbase.pb.GetMastersRequest) returns (.hbase.pb.GetMastersResponse);</code>
        *
        * <pre>
        **
-       * Get active master server name for this cluster.
+       * Get registered list of master servers in this cluster. List includes both active and backup
+       * masters.
        * </pre>
        */
-      public abstract void getActiveMaster(
+      public abstract void getMasters(
           com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest request,
-          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse> done);
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse> done);
 
       /**
        * <code>rpc GetMetaRegionLocations(.hbase.pb.GetMetaRegionLocationsRequest) returns (.hbase.pb.GetMetaRegionLocationsResponse);</code>
@@ -75095,11 +75859,11 @@ public final class MasterProtos {
         }
 
         @java.lang.Override
-        public  void getActiveMaster(
+        public  void getMasters(
             com.google.protobuf.RpcController controller,
-            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest request,
-            com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse> done) {
-          impl.getActiveMaster(controller, request, done);
+            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse> done) {
+          impl.getMasters(controller, request, done);
         }
 
         @java.lang.Override
@@ -75143,7 +75907,7 @@ public final class MasterProtos {
             case 0:
               return impl.getClusterId(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdRequest)request);
             case 1:
-              return impl.getActiveMaster(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest)request);
+              return impl.getMasters(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest)request);
             case 2:
               return impl.getMetaRegionLocations(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest)request);
             case 3:
@@ -75165,7 +75929,7 @@ public final class MasterProtos {
             case 0:
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdRequest.getDefaultInstance();
             case 1:
-              return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest.getDefaultInstance();
+              return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest.getDefaultInstance();
             case 2:
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest.getDefaultInstance();
             case 3:
@@ -75187,7 +75951,7 @@ public final class MasterProtos {
             case 0:
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdResponse.getDefaultInstance();
             case 1:
-              return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.getDefaultInstance();
+              return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.getDefaultInstance();
             case 2:
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse.getDefaultInstance();
             case 3:
@@ -75214,17 +75978,18 @@ public final class MasterProtos {
         com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdResponse> done);
 
     /**
-     * <code>rpc GetActiveMaster(.hbase.pb.GetActiveMasterRequest) returns (.hbase.pb.GetActiveMasterResponse);</code>
+     * <code>rpc GetMasters(.hbase.pb.GetMastersRequest) returns (.hbase.pb.GetMastersResponse);</code>
      *
      * <pre>
      **
-     * Get active master server name for this cluster.
+     * Get registered list of master servers in this cluster. List includes both active and backup
+     * masters.
      * </pre>
      */
-    public abstract void getActiveMaster(
+    public abstract void getMasters(
         com.google.protobuf.RpcController controller,
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest request,
-        com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse> done);
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse> done);
 
     /**
      * <code>rpc GetMetaRegionLocations(.hbase.pb.GetMetaRegionLocationsRequest) returns (.hbase.pb.GetMetaRegionLocationsResponse);</code>
@@ -75280,8 +76045,8 @@ public final class MasterProtos {
               done));
           return;
         case 1:
-          this.getActiveMaster(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest)request,
-            com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse>specializeCallback(
+          this.getMasters(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse>specializeCallback(
               done));
           return;
         case 2:
@@ -75311,7 +76076,7 @@ public final class MasterProtos {
         case 0:
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdRequest.getDefaultInstance();
         case 1:
-          return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest.getDefaultInstance();
+          return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest.getDefaultInstance();
         case 2:
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest.getDefaultInstance();
         case 3:
@@ -75333,7 +76098,7 @@ public final class MasterProtos {
         case 0:
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdResponse.getDefaultInstance();
         case 1:
-          return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.getDefaultInstance();
+          return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.getDefaultInstance();
         case 2:
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse.getDefaultInstance();
         case 3:
@@ -75374,19 +76139,19 @@ public final class MasterProtos {
             org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdResponse.getDefaultInstance()));
       }
 
-      public  void getActiveMaster(
+      public  void getMasters(
           com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest request,
-          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse> done) {
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse> done) {
         channel.callMethod(
           getDescriptor().getMethods().get(1),
           controller,
           request,
-          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.getDefaultInstance(),
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.getDefaultInstance(),
           com.google.protobuf.RpcUtil.generalizeCallback(
             done,
-            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.class,
-            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.getDefaultInstance()));
+            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.class,
+            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.getDefaultInstance()));
       }
 
       public  void getMetaRegionLocations(
@@ -75431,9 +76196,9 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdRequest request)
           throws com.google.protobuf.ServiceException;
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse getActiveMaster(
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse getMasters(
           com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest request)
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest request)
           throws com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse getMetaRegionLocations(
@@ -75466,15 +76231,15 @@ public final class MasterProtos {
       }
 
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse getActiveMaster(
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse getMasters(
           com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterRequest request)
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest request)
           throws com.google.protobuf.ServiceException {
-        return (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse) channel.callBlockingMethod(
+        return (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(1),
           controller,
           request,
-          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetActiveMasterResponse.getDefaultInstance());
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse.getDefaultInstance());
       }
 
 
@@ -76142,15 +76907,20 @@ public final class MasterProtos {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_GetClusterIdResponse_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_GetActiveMasterRequest_descriptor;
+    internal_static_hbase_pb_GetMastersRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_GetMastersRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_GetMastersResponseEntry_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_GetActiveMasterRequest_fieldAccessorTable;
+      internal_static_hbase_pb_GetMastersResponseEntry_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_GetActiveMasterResponse_descriptor;
+    internal_static_hbase_pb_GetMastersResponse_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_GetActiveMasterResponse_fieldAccessorTable;
+      internal_static_hbase_pb_GetMastersResponse_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetMetaRegionLocationsRequest_descriptor;
   private static
@@ -76390,174 +77160,176 @@ public final class MasterProtos {
       "anupEnabledRequest\"3\n IsSnapshotCleanupE" +
       "nabledResponse\022\017\n\007enabled\030\001 \002(\010\"\025\n\023GetCl" +
       "usterIdRequest\"*\n\024GetClusterIdResponse\022\022",
-      "\n\ncluster_id\030\001 \001(\t\"\030\n\026GetActiveMasterReq" +
-      "uest\"D\n\027GetActiveMasterResponse\022)\n\013serve" +
-      "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\"\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" +
+      "\n\ncluster_id\030\001 \001(\t\"\023\n\021GetMastersRequest\"" +
+      "W\n\027GetMastersResponseEntry\022)\n\013server_nam" +
+      "e\030\001 \002(\0132\024.hbase.pb.ServerName\022\021\n\tis_acti" +
+      "ve\030\002 \002(\010\"O\n\022GetMastersResponse\0229\n\016master" +
+      "_servers\030\001 \003(\0132!.hbase.pb.GetMastersResp" +
+      "onseEntry\"\037\n\035GetMetaRegionLocationsReque" +
+      "st\"R\n\036GetMetaRegionLocationsResponse\0220\n\016" +
+      "meta_locations\030\001 \003(\0132\030.hbase.pb.RegionLo" +
+      "cation\"\025\n\023GetNumLiveRSRequest\"2\n\024GetNumL" +
+      "iveRSResponse\022\032\n\022num_region_servers\030\001 \002(",
+      "\005*(\n\020MasterSwitchType\022\t\n\005SPLIT\020\000\022\t\n\005MERG" +
+      "E\020\0012\363.\n\rMasterService\022e\n\024GetSchemaAlterS" +
+      "tatus\022%.hbase.pb.GetSchemaAlterStatusReq" +
+      "uest\032&.hbase.pb.GetSchemaAlterStatusResp" +
+      "onse\022b\n\023GetTableDescriptors\022$.hbase.pb.G" +
+      "etTableDescriptorsRequest\032%.hbase.pb.Get" +
+      "TableDescriptorsResponse\022P\n\rGetTableName" +
+      "s\022\036.hbase.pb.GetTableNamesRequest\032\037.hbas" +
+      "e.pb.GetTableNamesResponse\022Y\n\020GetCluster" +
+      "Status\022!.hbase.pb.GetClusterStatusReques",
+      "t\032\".hbase.pb.GetClusterStatusResponse\022V\n" +
+      "\017IsMasterRunning\022 .hbase.pb.IsMasterRunn" +
+      "ingRequest\032!.hbase.pb.IsMasterRunningRes" +
+      "ponse\022D\n\tAddColumn\022\032.hbase.pb.AddColumnR" +
+      "equest\032\033.hbase.pb.AddColumnResponse\022M\n\014D" +
+      "eleteColumn\022\035.hbase.pb.DeleteColumnReque" +
+      "st\032\036.hbase.pb.DeleteColumnResponse\022M\n\014Mo" +
+      "difyColumn\022\035.hbase.pb.ModifyColumnReques" +
+      "t\032\036.hbase.pb.ModifyColumnResponse\022G\n\nMov" +
+      "eRegion\022\033.hbase.pb.MoveRegionRequest\032\034.h",
+      "base.pb.MoveRegionResponse\022k\n\026DispatchMe" +
+      "rgingRegions\022\'.hbase.pb.DispatchMergingR" +
+      "egionsRequest\032(.hbase.pb.DispatchMerging" +
+      "RegionsResponse\022M\n\014AssignRegion\022\035.hbase." +
+      "pb.AssignRegionRequest\032\036.hbase.pb.Assign" +
+      "RegionResponse\022S\n\016UnassignRegion\022\037.hbase" +
+      ".pb.UnassignRegionRequest\032 .hbase.pb.Una" +
+      "ssignRegionResponse\022P\n\rOfflineRegion\022\036.h" +
+      "base.pb.OfflineRegionRequest\032\037.hbase.pb." +
+      "OfflineRegionResponse\022J\n\013DeleteTable\022\034.h",
+      "base.pb.DeleteTableRequest\032\035.hbase.pb.De" +
+      "leteTableResponse\022P\n\rtruncateTable\022\036.hba" +
+      "se.pb.TruncateTableRequest\032\037.hbase.pb.Tr" +
+      "uncateTableResponse\022J\n\013EnableTable\022\034.hba" +
+      "se.pb.EnableTableRequest\032\035.hbase.pb.Enab" +
+      "leTableResponse\022M\n\014DisableTable\022\035.hbase." +
+      "pb.DisableTableRequest\032\036.hbase.pb.Disabl" +
+      "eTableResponse\022J\n\013ModifyTable\022\034.hbase.pb" +
+      ".ModifyTableRequest\032\035.hbase.pb.ModifyTab" +
+      "leResponse\022J\n\013CreateTable\022\034.hbase.pb.Cre",
+      "ateTableRequest\032\035.hbase.pb.CreateTableRe" +
+      "sponse\022A\n\010Shutdown\022\031.hbase.pb.ShutdownRe" +
+      "quest\032\032.hbase.pb.ShutdownResponse\022G\n\nSto" +
+      "pMaster\022\033.hbase.pb.StopMasterRequest\032\034.h" +
+      "base.pb.StopMasterResponse\022h\n\031IsMasterIn" +
+      "MaintenanceMode\022$.hbase.pb.IsInMaintenan" +
+      "ceModeRequest\032%.hbase.pb.IsInMaintenance" +
+      "ModeResponse\022>\n\007Balance\022\030.hbase.pb.Balan" +
+      "ceRequest\032\031.hbase.pb.BalanceResponse\022_\n\022" +
+      "SetBalancerRunning\022#.hbase.pb.SetBalance",
+      "rRunningRequest\032$.hbase.pb.SetBalancerRu" +
+      "nningResponse\022\\\n\021IsBalancerEnabled\022\".hba" +
+      "se.pb.IsBalancerEnabledRequest\032#.hbase.p" +
+      "b.IsBalancerEnabledResponse\022k\n\026SetSplitO" +
+      "rMergeEnabled\022\'.hbase.pb.SetSplitOrMerge" +
+      "EnabledRequest\032(.hbase.pb.SetSplitOrMerg" +
+      "eEnabledResponse\022h\n\025IsSplitOrMergeEnable" +
+      "d\022&.hbase.pb.IsSplitOrMergeEnabledReques" +
+      "t\032\'.hbase.pb.IsSplitOrMergeEnabledRespon" +
+      "se\022D\n\tNormalize\022\032.hbase.pb.NormalizeRequ",
+      "est\032\033.hbase.pb.NormalizeResponse\022e\n\024SetN" +
+      "ormalizerRunning\022%.hbase.pb.SetNormalize" +
+      "rRunningRequest\032&.hbase.pb.SetNormalizer" +
+      "RunningResponse\022b\n\023IsNormalizerEnabled\022$" +
+      ".hbase.pb.IsNormalizerEnabledRequest\032%.h" +
+      "base.pb.IsNormalizerEnabledResponse\022S\n\016R" +
+      "unCatalogScan\022\037.hbase.pb.RunCatalogScanR" +
+      "equest\032 .hbase.pb.RunCatalogScanResponse" +
+      "\022e\n\024EnableCatalogJanitor\022%.hbase.pb.Enab" +
+      "leCatalogJanitorRequest\032&.hbase.pb.Enabl",
+      "eCatalogJanitorResponse\022n\n\027IsCatalogJani" +
+      "torEnabled\022(.hbase.pb.IsCatalogJanitorEn" +
+      "abledRequest\032).hbase.pb.IsCatalogJanitor" +
+      "EnabledResponse\022V\n\017RunCleanerChore\022 .hba" +
+      "se.pb.RunCleanerChoreRequest\032!.hbase.pb." +
+      "RunCleanerChoreResponse\022k\n\026SetCleanerCho" +
+      "reRunning\022\'.hbase.pb.SetCleanerChoreRunn" +
+      "ingRequest\032(.hbase.pb.SetCleanerChoreRun" +
+      "ningResponse\022h\n\025IsCleanerChoreEnabled\022&." +
+      "hbase.pb.IsCleanerChoreEnabledRequest\032\'.",
+      "hbase.pb.IsCleanerChoreEnabledResponse\022^" +
+      "\n\021ExecMasterService\022#.hbase.pb.Coprocess" +
+      "orServiceRequest\032$.hbase.pb.CoprocessorS" +
+      "erviceResponse\022A\n\010Snapshot\022\031.hbase.pb.Sn" +
+      "apshotRequest\032\032.hbase.pb.SnapshotRespons" +
+      "e\022h\n\025GetCompletedSnapshots\022&.hbase.pb.Ge" +
+      "tCompletedSnapshotsRequest\032\'.hbase.pb.Ge" +
+      "tCompletedSnapshotsResponse\022S\n\016DeleteSna" +
+      "pshot\022\037.hbase.pb.DeleteSnapshotRequest\032 " +
+      ".hbase.pb.DeleteSnapshotResponse\022S\n\016IsSn",
+      "apshotDone\022\037.hbase.pb.IsSnapshotDoneRequ" +
+      "est\032 .hbase.pb.IsSnapshotDoneResponse\022V\n" +
+      "\017RestoreSnapshot\022 .hbase.pb.RestoreSnaps" +
+      "hotRequest\032!.hbase.pb.RestoreSnapshotRes" +
+      "ponse\022h\n\025IsRestoreSnapshotDone\022&.hbase.p" +
+      "b.IsRestoreSnapshotDoneRequest\032\'.hbase.p" +
+      "b.IsRestoreSnapshotDoneResponse\022P\n\rExecP" +
+      "rocedure\022\036.hbase.pb.ExecProcedureRequest" +
+      "\032\037.hbase.pb.ExecProcedureResponse\022W\n\024Exe" +
+      "cProcedureWithRet\022\036.hbase.pb.ExecProcedu",
+      "reRequest\032\037.hbase.pb.ExecProcedureRespon" +
+      "se\022V\n\017IsProcedureDone\022 .hbase.pb.IsProce" +
+      "dureDoneRequest\032!.hbase.pb.IsProcedureDo" +
+      "neResponse\022V\n\017ModifyNamespace\022 .hbase.pb" +
+      ".ModifyNamespaceRequest\032!.hbase.pb.Modif" +
+      "yNamespaceResponse\022V\n\017CreateNamespace\022 ." +
+      "hbase.pb.CreateNamespaceRequest\032!.hbase." +
+      "pb.CreateNamespaceResponse\022V\n\017DeleteName" +
+      "space\022 .hbase.pb.DeleteNamespaceRequest\032" +
+      "!.hbase.pb.DeleteNamespaceResponse\022k\n\026Ge",
+      "tNamespaceDescriptor\022\'.hbase.pb.GetNames" +
+      "paceDescriptorRequest\032(.hbase.pb.GetName" +
+      "spaceDescriptorResponse\022q\n\030ListNamespace" +
+      "Descriptors\022).hbase.pb.ListNamespaceDesc" +
+      "riptorsRequest\032*.hbase.pb.ListNamespaceD" +
+      "escriptorsResponse\022\206\001\n\037ListTableDescript" +
+      "orsByNamespace\0220.hbase.pb.ListTableDescr" +
+      "iptorsByNamespaceRequest\0321.hbase.pb.List" +
+      "TableDescriptorsByNamespaceResponse\022t\n\031L" +
+      "istTableNamesByNamespace\022*.hbase.pb.List",
+      "TableNamesByNamespaceRequest\032+.hbase.pb." +
+      "ListTableNamesByNamespaceResponse\022A\n\010Set" +
+      "Quota\022\031.hbase.pb.SetQuotaRequest\032\032.hbase" +
+      ".pb.SetQuotaResponse\022x\n\037getLastMajorComp" +
+      "actionTimestamp\022).hbase.pb.MajorCompacti" +
+      "onTimestampRequest\032*.hbase.pb.MajorCompa" +
+      "ctionTimestampResponse\022\212\001\n(getLastMajorC" +
+      "ompactionTimestampForRegion\0222.hbase.pb.M" +
+      "ajorCompactionTimestampForRegionRequest\032" +
+      "*.hbase.pb.MajorCompactionTimestampRespo",
+      "nse\022_\n\022getProcedureResult\022#.hbase.pb.Get" +
+      "ProcedureResultRequest\032$.hbase.pb.GetPro" +
+      "cedureResultResponse\022h\n\027getSecurityCapab" +
+      "ilities\022%.hbase.pb.SecurityCapabilitiesR" +
+      "equest\032&.hbase.pb.SecurityCapabilitiesRe" +
+      "sponse\022S\n\016AbortProcedure\022\037.hbase.pb.Abor" +
+      "tProcedureRequest\032 .hbase.pb.AbortProced" +
+      "ureResponse\022S\n\016ListProcedures\022\037.hbase.pb" +
+      ".ListProceduresRequest\032 .hbase.pb.ListPr" +
+      "oceduresResponse\022Y\n\020ClearDeadServers\022!.h",
+      "base.pb.ClearDeadServersRequest\032\".hbase." +
+      "pb.ClearDeadServersResponse\022S\n\016ListNames" +
+      "paces\022\037.hbase.pb.ListNamespacesRequest\032 " +
+      ".hbase.pb.ListNamespacesResponse\022b\n\025Swit" +
+      "chSnapshotCleanup\022#.hbase.pb.SetSnapshot" +
+      "CleanupRequest\032$.hbase.pb.SetSnapshotCle" +
+      "anupResponse\022q\n\030IsSnapshotCleanupEnabled" +
+      "\022).hbase.pb.IsSnapshotCleanupEnabledRequ" +
+      "est\032*.hbase.pb.IsSnapshotCleanupEnabledR" +
+      "esponse\022P\n\rGetTableState\022\036.hbase.pb.GetT",
+      "ableStateRequest\032\037.hbase.pb.GetTableStat" +
+      "eResponse2\347\002\n\021ClientMetaService\022M\n\014GetCl" +
+      "usterId\022\035.hbase.pb.GetClusterIdRequest\032\036" +
+      ".hbase.pb.GetClusterIdResponse\022G\n\nGetMas" +
+      "ters\022\033.hbase.pb.GetMastersRequest\032\034.hbas" +
+      "e.pb.GetMastersResponse\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" +
+      "GetNumLiveRSRequest\032\036.hbase.pb.GetNumLiv",
       "eRSResponseBB\n*org.apache.hadoop.hbase.p" +
       "rotobuf.generatedB\014MasterProtosH\001\210\001\001\240\001\001"
     };
@@ -77328,38 +78100,44 @@ public final class MasterProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetClusterIdResponse_descriptor,
               new java.lang.String[] { "ClusterId", });
-          internal_static_hbase_pb_GetActiveMasterRequest_descriptor =
+          internal_static_hbase_pb_GetMastersRequest_descriptor =
             getDescriptor().getMessageTypes().get(127);
-          internal_static_hbase_pb_GetActiveMasterRequest_fieldAccessorTable = new
+          internal_static_hbase_pb_GetMastersRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_hbase_pb_GetActiveMasterRequest_descriptor,
+              internal_static_hbase_pb_GetMastersRequest_descriptor,
               new java.lang.String[] { });
-          internal_static_hbase_pb_GetActiveMasterResponse_descriptor =
+          internal_static_hbase_pb_GetMastersResponseEntry_descriptor =
             getDescriptor().getMessageTypes().get(128);
-          internal_static_hbase_pb_GetActiveMasterResponse_fieldAccessorTable = new
+          internal_static_hbase_pb_GetMastersResponseEntry_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_hbase_pb_GetActiveMasterResponse_descriptor,
-              new java.lang.String[] { "ServerName", });
-          internal_static_hbase_pb_GetMetaRegionLocationsRequest_descriptor =
+              internal_static_hbase_pb_GetMastersResponseEntry_descriptor,
+              new java.lang.String[] { "ServerName", "IsActive", });
+          internal_static_hbase_pb_GetMastersResponse_descriptor =
             getDescriptor().getMessageTypes().get(129);
+          internal_static_hbase_pb_GetMastersResponse_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_GetMastersResponse_descriptor,
+              new java.lang.String[] { "MasterServers", });
+          internal_static_hbase_pb_GetMetaRegionLocationsRequest_descriptor =
+            getDescriptor().getMessageTypes().get(130);
           internal_static_hbase_pb_GetMetaRegionLocationsRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetMetaRegionLocationsRequest_descriptor,
               new java.lang.String[] { });
           internal_static_hbase_pb_GetMetaRegionLocationsResponse_descriptor =
-            getDescriptor().getMessageTypes().get(130);
+            getDescriptor().getMessageTypes().get(131);
           internal_static_hbase_pb_GetMetaRegionLocationsResponse_fieldAccessorTable = new
             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);
+            getDescriptor().getMessageTypes().get(132);
           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);
+            getDescriptor().getMessageTypes().get(133);
           internal_static_hbase_pb_GetNumLiveRSResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_GetNumLiveRSResponse_descriptor,
diff --git a/hbase-protocol/src/main/protobuf/Master.proto b/hbase-protocol/src/main/protobuf/Master.proto
index bf16dd5..be4c66e 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -922,12 +922,15 @@ message GetClusterIdResponse {
   optional string cluster_id = 1;
 }
 
-/** Request and response to get the currently active master name for this cluster */
-message GetActiveMasterRequest {
+/** Request and response to get the current list of all registers master servers */
+message GetMastersRequest {
 }
-message GetActiveMasterResponse {
-  /** Not set if an active master could not be determined. */
-  optional ServerName server_name = 1;
+message GetMastersResponseEntry {
+    required ServerName server_name = 1;
+    required bool is_active = 2;
+}
+message GetMastersResponse {
+    repeated GetMastersResponseEntry master_servers = 1;
 }
 
 /** Request and response to get the current list of meta region locations */
@@ -955,9 +958,10 @@ service ClientMetaService {
   rpc GetClusterId(GetClusterIdRequest) returns(GetClusterIdResponse);
 
   /**
-   * Get active master server name for this cluster.
+   * Get registered list of master servers in this cluster. List includes both active and backup
+   * masters.
    */
-  rpc GetActiveMaster(GetActiveMasterRequest) returns(GetActiveMasterResponse);
+  rpc GetMasters(GetMastersRequest) returns(GetMastersResponse);
 
   /**
    * Get current meta replicas' region locations.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
index d92a48e..61ace78 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
@@ -17,7 +17,10 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hbase.master;
+import com.google.common.collect.ImmutableList;
 import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
@@ -35,9 +38,10 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
- * Handles everything on master-side related to master election.
+ * Handles everything on master-side related to master election. Keeps track of
+ * currently active master and registered backup masters.
  *
- * <p>Listens and responds to ZooKeeper notifications on the master znode,
+ * <p>Listens and responds to ZooKeeper notifications on the master znodes,
  * both <code>nodeCreated</code> and <code>nodeDeleted</code>.
  *
  * <p>Contains blocking methods which will hold up backup masters, waiting
@@ -64,18 +68,23 @@ public class ActiveMasterManager extends ZooKeeperListener {
   // Active master's server name. Invalidated anytime active master changes (based on ZK
   // notifications) and lazily fetched on-demand.
   // ServerName is immutable, so we don't need heavy synchronization around it.
-  private volatile ServerName activeMasterServerName;
+  volatile ServerName activeMasterServerName;
+  // Registered backup masters. List is kept up to date based on ZK change notifications to
+  // backup znode.
+  private volatile ImmutableList<ServerName> backupMasters;
 
   /**
    * @param watcher ZK watcher
    * @param sn ServerName
    * @param master In an instance of a Master.
    */
-  ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master) {
+  ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master)
+      throws InterruptedIOException {
     super(watcher);
     watcher.registerListener(this);
     this.sn = sn;
     this.master = master;
+    updateBackupMasters();
   }
 
   // will be set after jetty server is started
@@ -89,8 +98,18 @@ public class ActiveMasterManager extends ZooKeeperListener {
   }
 
   @Override
-  public void nodeDeleted(String path) {
+  public void nodeChildrenChanged(String path) {
+    if (path.equals(watcher.backupMasterAddressesZNode)) {
+      try {
+        updateBackupMasters();
+      } catch (InterruptedIOException ioe) {
+        LOG.error("Error updating backup masters", ioe);
+      }
+    }
+  }
 
+  @Override
+  public void nodeDeleted(String path) {
     // We need to keep track of the cluster's shutdown status while
     // we wait on the current master. We consider that, if the cluster
     // was already in a "shutdown" state when we started, that this master
@@ -101,7 +120,6 @@ public class ActiveMasterManager extends ZooKeeperListener {
     if(path.equals(watcher.clusterStateZNode) && !master.isStopped()) {
       clusterShutDown.set(true);
     }
-
     handle(path);
   }
 
@@ -111,6 +129,11 @@ public class ActiveMasterManager extends ZooKeeperListener {
     }
   }
 
+  private void updateBackupMasters() throws InterruptedIOException {
+    backupMasters =
+        ImmutableList.copyOf(MasterAddressTracker.getBackupMastersAndRenewWatch(watcher));
+  }
+
   /**
    * Fetches the active master's ServerName from zookeeper.
    */
@@ -320,4 +343,11 @@ public class ActiveMasterManager extends ZooKeeperListener {
       LOG.error(this.watcher.prefix("Error deleting our own master address node"), e);
     }
   }
+
+  /**
+   * @return list of registered backup masters.
+   */
+  public List<ServerName> getBackupMasters() {
+    return backupMasters;
+  }
 }
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 2bb4362..1a2f52b 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
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Service;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.lang.reflect.Constructor;
@@ -27,7 +29,6 @@ import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -86,7 +87,6 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.http.InfoServer;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
@@ -189,12 +189,8 @@ import org.mortbay.jetty.servlet.ServletHolder;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Service;
 
 /**
- * HMaster is the "master server" for HBase. An HBase cluster has one active
- * master.  If many masters are started, all compete.  Whichever wins goes on to
  * run the cluster.  All others park themselves in their constructor until
  * master or cluster shutdown or until the active master loses its lease in
  * zookeeper.  Thereafter, all running master jostle to take over master role.
@@ -2548,56 +2544,14 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    */
   public ClusterStatus getClusterStatusWithoutCoprocessor() throws InterruptedIOException {
     // Build Set of backup masters from ZK nodes
-    List<String> backupMasterStrings;
-    try {
-      backupMasterStrings = ZKUtil.listChildrenNoWatch(this.zooKeeper,
-        this.zooKeeper.backupMasterAddressesZNode);
-    } catch (KeeperException e) {
-      LOG.warn(this.zooKeeper.prefix("Unable to list backup servers"), e);
-      backupMasterStrings = null;
-    }
-
-    List<ServerName> backupMasters = null;
-    if (backupMasterStrings != null && !backupMasterStrings.isEmpty()) {
-      backupMasters = new ArrayList<ServerName>(backupMasterStrings.size());
-      for (String s: backupMasterStrings) {
-        try {
-          byte [] bytes;
-          try {
-            bytes = ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode(
-                this.zooKeeper.backupMasterAddressesZNode, s));
-          } catch (InterruptedException e) {
-            throw new InterruptedIOException();
-          }
-          if (bytes != null) {
-            ServerName sn;
-            try {
-              sn = ServerName.parseFrom(bytes);
-            } catch (DeserializationException e) {
-              LOG.warn("Failed parse, skipping registering backup server", e);
-              continue;
-            }
-            backupMasters.add(sn);
-          }
-        } catch (KeeperException e) {
-          LOG.warn(this.zooKeeper.prefix("Unable to get information about " +
-                   "backup servers"), e);
-        }
-      }
-      Collections.sort(backupMasters, new Comparator<ServerName>() {
-        @Override
-        public int compare(ServerName s1, ServerName s2) {
-          return s1.getServerName().compareTo(s2.getServerName());
-        }});
-    }
-
+    List<ServerName> backupMasters = getBackupMasters();
     String clusterId = fileSystemManager != null ?
-      fileSystemManager.getClusterId().toString() : null;
+        fileSystemManager.getClusterId().toString() : null;
     Set<RegionState> regionsInTransition = assignmentManager != null ?
-      assignmentManager.getRegionStates().getRegionsInTransition() : null;
+        assignmentManager.getRegionStates().getRegionsInTransition() : null;
     String[] coprocessors = cpHost != null ? getMasterCoprocessors() : null;
     boolean balancerOn = loadBalancerTracker != null ?
-      loadBalancerTracker.isBalancerOn() : false;
+        loadBalancerTracker.isBalancerOn() : false;
     Map<ServerName, ServerLoad> onlineServers = null;
     Set<ServerName> deadServers = null;
     if (serverManager != null) {
@@ -2605,8 +2559,12 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       onlineServers = serverManager.getOnlineServers();
     }
     return new ClusterStatus(VersionInfo.getVersion(), clusterId,
-      onlineServers, deadServers, serverName, backupMasters,
-      regionsInTransition, coprocessors, balancerOn);
+        onlineServers, deadServers, serverName, backupMasters,
+        regionsInTransition, coprocessors, balancerOn);
+  }
+
+  List<ServerName> getBackupMasters() {
+    return activeMasterManager.getBackupMasters();
   }
 
   /**
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 e4eb654..963b94e 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
@@ -97,14 +97,15 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableReques
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
-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.GetClusterStatusRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMastersResponseEntry;
 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.GetNamespaceDescriptorRequest;
@@ -1804,12 +1805,19 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
-  public GetActiveMasterResponse getActiveMaster(RpcController rpcController,
-      GetActiveMasterRequest request) throws ServiceException {
-    GetActiveMasterResponse.Builder resp = GetActiveMasterResponse.newBuilder();
+  public GetMastersResponse getMasters(RpcController rpcController, GetMastersRequest request)
+      throws ServiceException {
+    GetMastersResponse.Builder resp = GetMastersResponse.newBuilder();
+    // Active master
     ServerName serverName = master.getActiveMaster();
     if (serverName != null) {
-      resp.setServerName(ProtobufUtil.toServerName(serverName));
+      resp.addMasterServers(GetMastersResponseEntry.newBuilder()
+          .setServerName(ProtobufUtil.toServerName(serverName)).setIsActive(true).build());
+    }
+    // Backup masters
+    for (ServerName backupMaster: master.getBackupMasters()) {
+      resp.addMasterServers(GetMastersResponseEntry.newBuilder().setServerName(
+          ProtobufUtil.toServerName(backupMaster)).setIsActive(false).build());
     }
     return resp.build();
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterAddressRefresher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterAddressRefresher.java
new file mode 100644
index 0000000..7e2f2f7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterAddressRefresher.java
@@ -0,0 +1,176 @@
+/*
+ * 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 com.google.common.util.concurrent.Uninterruptibles;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ClientTests.class, SmallTests.class})
+public class TestMasterAddressRefresher {
+
+  static class DummyConnection implements Connection {
+    private final Configuration conf;
+
+    DummyConnection(Configuration conf) {
+      this.conf = conf;
+    }
+
+    @Override
+    public Configuration getConfiguration() {
+      return conf;
+    }
+
+    @Override
+    public Table getTable(TableName tableName) throws IOException {
+      return null;
+    }
+
+    @Override
+    public Table getTable(TableName tableName, ExecutorService pool) throws IOException {
+      return null;
+    }
+
+    @Override
+    public BufferedMutator getBufferedMutator(TableName tableName) throws IOException {
+      return null;
+    }
+
+    @Override
+    public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException {
+      return null;
+    }
+
+    @Override
+    public RegionLocator getRegionLocator(TableName tableName) throws IOException {
+      return null;
+    }
+
+    @Override
+    public Admin getAdmin() throws IOException {
+      return null;
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+
+    @Override
+    public boolean isClosed() {
+      return false;
+    }
+
+    @Override
+    public void abort(String why, Throwable e) {
+
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+  }
+
+  private static class DummyMasterRegistry extends MasterRegistry {
+
+    private final AtomicInteger getMastersCallCounter = new AtomicInteger(0);
+    private final List<Long> callTimeStamps = new ArrayList<>();
+
+    @Override
+    public void init(Connection connection) throws IOException {
+      super.init(connection);
+    }
+
+    @Override
+    List<ServerName> getMasters() {
+      getMastersCallCounter.incrementAndGet();
+      callTimeStamps.add(EnvironmentEdgeManager.currentTime());
+      return new ArrayList<>();
+    }
+
+    public int getMastersCount() {
+      return getMastersCallCounter.get();
+    }
+
+    public List<Long> getCallTimeStamps() {
+      return callTimeStamps;
+    }
+  }
+
+  @Test
+  public void testPeriodicMasterEndPointRefresh() throws IOException {
+    Configuration conf = HBaseConfiguration.create();
+    // Refresh every 1 second.
+    conf.setLong(MasterAddressRefresher.PERIODIC_REFRESH_INTERVAL_SECS, 1);
+    conf.setLong(MasterAddressRefresher.MIN_SECS_BETWEEN_REFRESHES, 0);
+    final DummyMasterRegistry registry = new DummyMasterRegistry();
+    registry.init(new DummyConnection(conf));
+    // Wait for > 3 seconds to see that at least 3 getMasters() RPCs have been made.
+    Waiter.waitFor(
+        conf, 5000, new Waiter.Predicate<Exception>() {
+          @Override
+          public boolean evaluate() throws Exception {
+            return registry.getMastersCount() > 3;
+          }
+        });
+  }
+
+  @Test
+  public void testDurationBetweenRefreshes() throws IOException {
+    Configuration conf = HBaseConfiguration.create();
+    // Disable periodic refresh
+    conf.setLong(MasterAddressRefresher.PERIODIC_REFRESH_INTERVAL_SECS, Integer.MAX_VALUE);
+    // A minimum duration of 1s between refreshes
+    conf.setLong(MasterAddressRefresher.MIN_SECS_BETWEEN_REFRESHES, 1);
+    DummyMasterRegistry registry = new DummyMasterRegistry();
+    registry.init(new DummyConnection(conf));
+    // Issue a ton of manual refreshes.
+    for (int i = 0; i < 10000; i++) {
+      registry.masterAddressRefresher.refreshNow();
+      Uninterruptibles.sleepUninterruptibly(1, TimeUnit.MILLISECONDS);
+    }
+    // Overall wait time is 10000 ms, so the number of requests should be <=10
+    List<Long> callTimeStamps = registry.getCallTimeStamps();
+    // Actual calls to getMasters() should be much lower than the refresh count.
+    Assert.assertTrue(
+        String.valueOf(registry.getMastersCount()), registry.getMastersCount() <= 20);
+    Assert.assertTrue(callTimeStamps.size() > 0);
+    // Verify that the delta between subsequent RPCs is at least 1sec as configured.
+    for (int i = 1; i < callTimeStamps.size() - 1; i++) {
+      long delta = callTimeStamps.get(i) - callTimeStamps.get(i - 1);
+      // Few ms cushion to account for any env jitter.
+      Assert.assertTrue(callTimeStamps.toString(), delta > 990);
+    }
+  }
+}
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
index 07f0100..0695e4b 100644
--- 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
@@ -19,9 +19,15 @@ package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.HConstants.META_REPLICAS_NUM;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.protobuf.RpcController;
 import java.io.IOException;
+import java.net.SocketTimeoutException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -33,6 +39,7 @@ 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.exceptions.MasterRegistryFetchException;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -41,6 +48,10 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ClientMetaService;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterIdResponse;
+
 @Category({ MediumTests.class, ClientTests.class })
 public class TestMasterRegistry {
 
@@ -59,6 +70,20 @@ public class TestMasterRegistry {
     TEST_UTIL.shutdownMiniCluster();
   }
 
+  private static class ExceptionInjectorRegistry extends MasterRegistry {
+    @Override
+    public String getClusterId() throws IOException {
+      GetClusterIdResponse resp = doCall(new Callable<GetClusterIdResponse>() {
+        @Override
+        public GetClusterIdResponse call(ClientMetaService.Interface stub, RpcController controller)
+            throws IOException {
+          throw new SocketTimeoutException("Injected exception.");
+        }
+      });
+      return resp.getClusterId();
+    }
+  }
+
   /**
    * Generates a string of dummy master addresses in host:port format. Every other hostname won't
    * have a port number.
@@ -130,4 +155,82 @@ public class TestMasterRegistry {
       registry.close();
     }
   }
+
+  /**
+   * Tests that the list of masters configured in the MasterRegistry is dynamically refreshed in the
+   * event of errors.
+   */
+  @Test
+  public void testDynamicMasterConfigurationRefresh() throws Exception {
+    Configuration conf = TEST_UTIL.getConnection().getConfiguration();
+    String currentMasterAddrs = Preconditions.checkNotNull(conf.get(HConstants.MASTER_ADDRS_KEY));
+    HMaster activeMaster = TEST_UTIL.getHBaseCluster().getMaster();
+    // Add a non-working master
+    ServerName badServer = ServerName.valueOf("localhost", 1234, -1);
+    conf.set(HConstants.MASTER_ADDRS_KEY, badServer.toShortString() + "," + currentMasterAddrs);
+    // Do not limit the number of refreshes during the test run.
+    conf.setLong(MasterAddressRefresher.MIN_SECS_BETWEEN_REFRESHES, 0);
+    final ExceptionInjectorRegistry registry = new ExceptionInjectorRegistry();
+    try {
+      registry.init(TEST_UTIL.getConnection());
+      final ImmutableSet<String> masters = registry.getParsedMasterServers();
+      assertTrue(masters.contains(badServer.toString()));
+      // Make a registry RPC, this should trigger a refresh since one of the RPC fails.
+      try {
+        registry.getClusterId();
+      } catch (MasterRegistryFetchException e) {
+        // Expected.
+      }
+
+      // Wait for new set of masters to be populated.
+      TEST_UTIL.waitFor(5000,
+          new Waiter.Predicate<Exception>() {
+            @Override
+            public boolean evaluate() throws Exception {
+              return !registry.getParsedMasterServers().equals(masters);
+            }
+          });
+      // new set of masters should not include the bad server
+      final ImmutableSet<String> newMasters = registry.getParsedMasterServers();
+      // Bad one should be out.
+      assertEquals(3, newMasters.size());
+      assertFalse(newMasters.contains(badServer.toString()));
+      // Kill the active master
+      activeMaster.stopMaster();
+      TEST_UTIL.waitFor(10000,
+          new Waiter.Predicate<Exception>() {
+            @Override
+            public boolean evaluate() {
+              return TEST_UTIL.getMiniHBaseCluster().getLiveMasterThreads().size() == 2;
+            }
+          });
+      TEST_UTIL.getMiniHBaseCluster().waitForActiveAndReadyMaster(10000);
+      // Make a registry RPC, this should trigger a refresh since one of the RPC fails.
+      try {
+        registry.getClusterId();
+      } catch (MasterRegistryFetchException e) {
+        // Expected.
+      }
+      // Wait until the killed master de-registered.
+      TEST_UTIL.waitFor(10000, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return registry.getMasters().size() == 2;
+        }
+      });
+      TEST_UTIL.waitFor(20000, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return registry.getParsedMasterServers().size() == 2;
+        }
+      });
+      final ImmutableSet<String> newMasters2 = registry.getParsedMasterServers();
+      assertEquals(2, newMasters2.size());
+      assertFalse(newMasters2.contains(activeMaster.getServerName().toString()));
+    } finally {
+      registry.close();
+      // Reset the state, add a killed master.
+      TEST_UTIL.getMiniHBaseCluster().startMaster();
+    }
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
index 3144eeb..5e3106f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -21,10 +21,12 @@ package org.apache.hadoop.hbase.master;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.Semaphore;
 
 import org.apache.commons.logging.Log;
@@ -33,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
@@ -70,43 +73,41 @@ public class TestActiveMasterManager {
   }
 
   @Test public void testRestartMaster() throws IOException, KeeperException {
-    ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
-      "testActiveMasterManagerFromZK", null, true);
-    try {
-      ZKUtil.deleteNode(zk, zk.getMasterAddressZNode());
-      ZKUtil.deleteNode(zk, zk.clusterStateZNode);
-    } catch(KeeperException.NoNodeException nne) {}
-
-    // Create the master node with a dummy address
-    ServerName master = ServerName.valueOf("localhost", 1, System.currentTimeMillis());
-    // Should not have a master yet
-    DummyMaster dummyMaster = new DummyMaster(zk,master);
-    ClusterStatusTracker clusterStatusTracker =
-      dummyMaster.getClusterStatusTracker();
-    ActiveMasterManager activeMasterManager =
-      dummyMaster.getActiveMasterManager();
-    assertFalse(activeMasterManager.clusterHasActiveMaster.get());
-    assertNull(activeMasterManager.getActiveMasterServerName());
-
-    // First test becoming the active master uninterrupted
-    MonitoredTask status = Mockito.mock(MonitoredTask.class);
-    clusterStatusTracker.setClusterUp();
-
-    activeMasterManager.blockUntilBecomingActiveMaster(100, status);
-    assertTrue(activeMasterManager.clusterHasActiveMaster.get());
-    assertMaster(zk, master);
-    assertMaster(zk, activeMasterManager.getActiveMasterServerName());
-
-    // Now pretend master restart
-    DummyMaster secondDummyMaster = new DummyMaster(zk,master);
-    ActiveMasterManager secondActiveMasterManager =
-      secondDummyMaster.getActiveMasterManager();
-    assertFalse(secondActiveMasterManager.clusterHasActiveMaster.get());
-    activeMasterManager.blockUntilBecomingActiveMaster(100, status);
-    assertTrue(activeMasterManager.clusterHasActiveMaster.get());
-    assertMaster(zk, master);
-    assertMaster(zk, activeMasterManager.getActiveMasterServerName());
-    assertMaster(zk, secondActiveMasterManager.getActiveMasterServerName());
+    try (ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+        "testActiveMasterManagerFromZK", null, true)) {
+      try {
+        ZKUtil.deleteNode(zk, zk.getMasterAddressZNode());
+        ZKUtil.deleteNode(zk, zk.clusterStateZNode);
+      } catch (KeeperException.NoNodeException nne) {
+      }
+
+      // Create the master node with a dummy address
+      ServerName master = ServerName.valueOf("localhost", 1, System.currentTimeMillis());
+      // Should not have a master yet
+      DummyMaster dummyMaster = new DummyMaster(zk, master);
+      ClusterStatusTracker clusterStatusTracker =
+          dummyMaster.getClusterStatusTracker();
+      ActiveMasterManager activeMasterManager =
+          dummyMaster.getActiveMasterManager();
+      assertFalse(activeMasterManager.clusterHasActiveMaster.get());
+
+      // First test becoming the active master uninterrupted
+      MonitoredTask status = Mockito.mock(MonitoredTask.class);
+      clusterStatusTracker.setClusterUp();
+
+      activeMasterManager.blockUntilBecomingActiveMaster(100, status);
+      assertTrue(activeMasterManager.clusterHasActiveMaster.get());
+      assertMaster(zk, master);
+
+      // Now pretend master restart
+      DummyMaster secondDummyMaster = new DummyMaster(zk, master);
+      ActiveMasterManager secondActiveMasterManager =
+          secondDummyMaster.getActiveMasterManager();
+      assertFalse(secondActiveMasterManager.clusterHasActiveMaster.get());
+      activeMasterManager.blockUntilBecomingActiveMaster(100, status);
+      assertTrue(activeMasterManager.clusterHasActiveMaster.get());
+      assertMaster(zk, master);
+    }
   }
 
   /**
@@ -116,86 +117,126 @@ public class TestActiveMasterManager {
    */
   @Test
   public void testActiveMasterManagerFromZK() throws Exception {
-    ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
-      "testActiveMasterManagerFromZK", null, true);
-    try {
+    try (ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+        "testActiveMasterManagerFromZK", null, true)) {
+      try {
+        ZKUtil.deleteNode(zk, zk.getMasterAddressZNode());
+        ZKUtil.deleteNode(zk, zk.clusterStateZNode);
+      } catch (KeeperException.NoNodeException nne) {
+      }
+
+      // Create the master node with a dummy address
+      ServerName firstMasterAddress =
+          ServerName.valueOf("localhost", 1, System.currentTimeMillis());
+      ServerName secondMasterAddress =
+          ServerName.valueOf("localhost", 2, System.currentTimeMillis());
+
+      // Should not have a master yet
+      DummyMaster ms1 = new DummyMaster(zk, firstMasterAddress);
+      ActiveMasterManager activeMasterManager =
+          ms1.getActiveMasterManager();
+      assertFalse(activeMasterManager.clusterHasActiveMaster.get());
+
+      // First test becoming the active master uninterrupted
+      ClusterStatusTracker clusterStatusTracker =
+          ms1.getClusterStatusTracker();
+      clusterStatusTracker.setClusterUp();
+      activeMasterManager.blockUntilBecomingActiveMaster(100,
+          Mockito.mock(MonitoredTask.class));
+      assertTrue(activeMasterManager.clusterHasActiveMaster.get());
+      assertMaster(zk, firstMasterAddress);
+
+      // New manager will now try to become the active master in another thread
+      WaitToBeMasterThread t = new WaitToBeMasterThread(zk, secondMasterAddress);
+      t.start();
+      // Wait for this guy to figure out there is another active master
+      // Wait for 1 second at most
+      int sleeps = 0;
+      while (!t.manager.clusterHasActiveMaster.get() && sleeps < 100) {
+        Thread.sleep(10);
+        sleeps++;
+      }
+
+      // Both should see that there is an active master
+      assertTrue(activeMasterManager.clusterHasActiveMaster.get());
+      assertTrue(t.manager.clusterHasActiveMaster.get());
+      // But secondary one should not be the active master
+      assertFalse(t.isActiveMaster);
+
+      // Close the first server and delete it's master node
+      ms1.stop("stopping first server");
+
+      // Use a listener to capture when the node is actually deleted
+      NodeDeletionListener listener = new NodeDeletionListener(zk, zk.getMasterAddressZNode());
+      zk.registerListener(listener);
+
+      LOG.info("Deleting master node");
       ZKUtil.deleteNode(zk, zk.getMasterAddressZNode());
-      ZKUtil.deleteNode(zk, zk.clusterStateZNode);
-    } catch(KeeperException.NoNodeException nne) {}
-
-    // Create the master node with a dummy address
-    ServerName firstMasterAddress =
-        ServerName.valueOf("localhost", 1, System.currentTimeMillis());
-    ServerName secondMasterAddress =
-        ServerName.valueOf("localhost", 2, System.currentTimeMillis());
-
-    // Should not have a master yet
-    DummyMaster ms1 = new DummyMaster(zk,firstMasterAddress);
-    ActiveMasterManager activeMasterManager =
-      ms1.getActiveMasterManager();
-    assertFalse(activeMasterManager.clusterHasActiveMaster.get());
-    assertNull(activeMasterManager.getActiveMasterServerName());
-
-    // First test becoming the active master uninterrupted
-    ClusterStatusTracker clusterStatusTracker =
-      ms1.getClusterStatusTracker();
-    clusterStatusTracker.setClusterUp();
-    activeMasterManager.blockUntilBecomingActiveMaster(100,
-        Mockito.mock(MonitoredTask.class));
-    assertTrue(activeMasterManager.clusterHasActiveMaster.get());
-    assertMaster(zk, firstMasterAddress);
-    assertMaster(zk, activeMasterManager.getActiveMasterServerName());
-
-    // New manager will now try to become the active master in another thread
-    WaitToBeMasterThread t = new WaitToBeMasterThread(zk, secondMasterAddress);
-    t.start();
-    // Wait for this guy to figure out there is another active master
-    // Wait for 1 second at most
-    int sleeps = 0;
-    while(!t.manager.clusterHasActiveMaster.get() && sleeps < 100) {
-      Thread.sleep(10);
-      sleeps++;
-    }
 
-    // Both should see that there is an active master
-    assertTrue(activeMasterManager.clusterHasActiveMaster.get());
-    assertTrue(t.manager.clusterHasActiveMaster.get());
-    // But secondary one should not be the active master
-    assertFalse(t.isActiveMaster);
-    // Verify the active master ServerName is populated in standby master.
-    assertEquals(firstMasterAddress, t.manager.getActiveMasterServerName());
-
-    // Close the first server and delete it's master node
-    ms1.stop("stopping first server");
-
-    // Use a listener to capture when the node is actually deleted
-    NodeDeletionListener listener = new NodeDeletionListener(zk, zk.getMasterAddressZNode());
-    zk.registerListener(listener);
-
-    LOG.info("Deleting master node");
-    ZKUtil.deleteNode(zk, zk.getMasterAddressZNode());
-
-    // Wait for the node to be deleted
-    LOG.info("Waiting for active master manager to be notified");
-    listener.waitForDeletion();
-    LOG.info("Master node deleted");
-
-    // Now we expect the secondary manager to have and be the active master
-    // Wait for 1 second at most
-    sleeps = 0;
-    while(!t.isActiveMaster && sleeps < 100) {
-      Thread.sleep(10);
-      sleeps++;
-    }
-    LOG.debug("Slept " + sleeps + " times");
+      // Wait for the node to be deleted
+      LOG.info("Waiting for active master manager to be notified");
+      listener.waitForDeletion();
+      LOG.info("Master node deleted");
+
+      // Now we expect the secondary manager to have and be the active master
+      // Wait for 1 second at most
+      sleeps = 0;
+      while (!t.isActiveMaster && sleeps < 100) {
+        Thread.sleep(10);
+        sleeps++;
+      }
+      LOG.debug("Slept " + sleeps + " times");
 
-    assertTrue(t.manager.clusterHasActiveMaster.get());
-    assertTrue(t.isActiveMaster);
-    assertEquals(secondMasterAddress, t.manager.getActiveMasterServerName());
+      assertTrue(t.manager.clusterHasActiveMaster.get());
+      assertTrue(t.isActiveMaster);
 
-    LOG.info("Deleting master node");
+      LOG.info("Deleting master node");
+
+      ZKUtil.deleteNode(zk, zk.getMasterAddressZNode());
+    }
+  }
 
-    ZKUtil.deleteNode(zk, zk.getMasterAddressZNode());
+  @Test
+  public void testBackupMasterUpdates() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    try (ZooKeeperWatcher zk = new ZooKeeperWatcher(
+        conf, "testBackupMasterUpdates", null, true)) {
+      ServerName sn1 = ServerName.valueOf("localhost", 1, -1);
+      DummyMaster master1 = new DummyMaster(zk, sn1);
+      final ActiveMasterManager activeMasterManager = master1.getActiveMasterManager();
+      activeMasterManager.blockUntilBecomingActiveMaster(100,
+          Mockito.mock(MonitoredTask.class));
+      assertEquals(sn1, activeMasterManager.getActiveMasterServerName());
+      assertEquals(0, activeMasterManager.getBackupMasters().size());
+      // Add backup masters
+      final List<String> backupZNodes = new ArrayList<>();
+      for (int i = 1; i <= 10; i++) {
+        ServerName backupSn = ServerName.valueOf("localhost", 1000 + i, -1);
+        String backupZn = ZKUtil.joinZNode(zk.backupMasterAddressesZNode, backupSn.toString());
+        backupZNodes.add(backupZn);
+        MasterAddressTracker.setMasterAddress(zk, backupZn, backupSn, 1234);
+        TEST_UTIL.waitFor(10000,
+            new Waiter.Predicate<Exception>() {
+              @Override
+              public boolean evaluate() throws Exception {
+                return activeMasterManager.getBackupMasters().size() == backupZNodes.size();
+              }
+            });
+      }
+      // Remove backup masters
+      int numBackups = backupZNodes.size();
+      for (String backupZNode: backupZNodes) {
+        ZKUtil.deleteNode(zk, backupZNode);
+        final int currentBackups = --numBackups;
+        TEST_UTIL.waitFor(10000,
+            new Waiter.Predicate<Exception>() {
+              @Override
+              public boolean evaluate() throws Exception {
+                return activeMasterManager.getBackupMasters().size() == currentBackups;
+              }
+            });
+      }
+    }
   }
 
   /**
@@ -206,8 +247,8 @@ public class TestActiveMasterManager {
    * @throws IOException if an IO problem is encountered
    */
   private void assertMaster(ZooKeeperWatcher zk,
-      ServerName expectedAddress)
-  throws KeeperException, IOException {
+                            ServerName expectedAddress)
+      throws KeeperException, IOException {
     ServerName readAddress = MasterAddressTracker.getMasterAddress(zk);
     assertNotNull(readAddress);
     assertTrue(expectedAddress.equals(readAddress));
@@ -219,7 +260,8 @@ public class TestActiveMasterManager {
     DummyMaster dummyMaster;
     boolean isActiveMaster;
 
-    public WaitToBeMasterThread(ZooKeeperWatcher zk, ServerName address) {
+    public WaitToBeMasterThread(ZooKeeperWatcher zk, ServerName address)
+        throws InterruptedIOException {
       this.dummyMaster = new DummyMaster(zk,address);
       this.manager = this.dummyMaster.getActiveMasterManager();
       isActiveMaster = false;
@@ -267,13 +309,13 @@ public class TestActiveMasterManager {
     private ClusterStatusTracker clusterStatusTracker;
     private ActiveMasterManager activeMasterManager;
 
-    public DummyMaster(ZooKeeperWatcher zk, ServerName master) {
+    public DummyMaster(ZooKeeperWatcher zk, ServerName master) throws InterruptedIOException {
       this.clusterStatusTracker =
-        new ClusterStatusTracker(zk, this);
+          new ClusterStatusTracker(zk, this);
       clusterStatusTracker.start();
 
       this.activeMasterManager =
-        new ActiveMasterManager(zk, master, this);
+          new ActiveMasterManager(zk, master, this);
       zk.registerListener(activeMasterManager);
     }
 
@@ -338,4 +380,4 @@ public class TestActiveMasterManager {
       return null;
     }
   }
-}
+}
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClientMetaServiceRPCs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClientMetaServiceRPCs.java
index 458f891..013d2a4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClientMetaServiceRPCs.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClientMetaServiceRPCs.java
@@ -45,8 +45,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 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;
@@ -112,25 +110,6 @@ public class TestClientMetaServiceRPCs {
   }
 
   /**
-   * Verifies the active master ServerName as seen by all masters.
-   */
-  @Test public void TestActiveMaster() throws Exception {
-    HBaseRpcController rpcController = getRpcController();
-    ServerName activeMaster = TEST_UTIL.getMiniHBaseCluster().getMaster().getServerName();
-    int rpcCount = 0;
-    for (JVMClusterUtil.MasterThread masterThread:
-        TEST_UTIL.getMiniHBaseCluster().getMasterThreads()) {
-      ClientMetaService.BlockingInterface stub =
-          getMasterStub(masterThread.getMaster().getServerName());
-      GetActiveMasterResponse resp =
-          stub.getActiveMaster(rpcController, GetActiveMasterRequest.getDefaultInstance());
-      assertEquals(activeMaster, ProtobufUtil.toServerName(resp.getServerName()));
-      rpcCount++;
-    }
-    assertEquals(MASTER_COUNT, rpcCount);
-  }
-
-  /**
    * Verifies that the meta region locations RPC returns consistent results across all masters.
    */
   @Test public void TestMetaLocations() throws Exception {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
index fcbe0a6..a5b3d49 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -1138,10 +1139,16 @@ public class TestMasterFailover {
 
     // Check that ClusterStatus reports the correct active and backup masters
     assertNotNull(active);
+    final HMaster finalActive = active;
+    TEST_UTIL.waitFor(10000, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        ClusterStatus status = finalActive.getClusterStatus();
+        return status.getBackupMastersSize() == 1 && status.getBackupMasters().size() == 1;
+      }
+    });
     status = active.getClusterStatus();
     assertTrue(status.getMaster().equals(activeName));
-    assertEquals(1, status.getBackupMastersSize());
-    assertEquals(1, status.getBackupMasters().size());
 
     // kill the active master
     LOG.debug("\n\nStopping the active master " + active.getServerName() + "\n");
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java
index fd8c4dc..cdb6af5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
+import java.util.List;
 import java.util.concurrent.Semaphore;
 
 import org.apache.commons.logging.Log;
@@ -33,6 +34,7 @@ import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -45,10 +47,19 @@ public class TestMasterAddressTracker {
   private static final Log LOG = LogFactory.getLog(TestMasterAddressTracker.class);
 
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  // Cleaned up after each unit test.
+  private static ZooKeeperWatcher zk;
 
   @Rule
   public TestName name = new TestName();
 
+  @After
+  public void cleanUp() {
+    if (zk != null) {
+      zk.close();
+    }
+  }
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.startMiniZKCluster();
@@ -79,9 +90,10 @@ public class TestMasterAddressTracker {
    */
   private MasterAddressTracker setupMasterTracker(final ServerName sn, final int infoPort)
       throws Exception {
-    ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+    zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
         name.getMethodName(), null);
     ZKUtil.createAndFailSilent(zk, zk.baseZNode);
+    ZKUtil.createAndFailSilent(zk, zk.backupMasterAddressesZNode);
 
     // Should not have a master yet
     MasterAddressTracker addressTracker = new MasterAddressTracker(zk, null);
@@ -155,6 +167,29 @@ public class TestMasterAddressTracker {
     assertEquals("Should receive 0 for backup not found.", 0, addressTracker.getMasterInfoPort());
   }
 
+  @Test
+  public void testBackupMasters() throws Exception {
+    final ServerName sn = ServerName.valueOf("localhost", 5678, System.currentTimeMillis());
+    final MasterAddressTracker addressTracker = setupMasterTracker(sn, 1111);
+    assertTrue(addressTracker.hasMaster());
+    ServerName activeMaster = addressTracker.getMasterAddress();
+    assertEquals(activeMaster, sn);
+    // No current backup masters
+    List<ServerName> backupMasters = MasterAddressTracker.getBackupMastersAndRenewWatch(zk);
+    assertEquals(0, backupMasters.size());
+    ServerName backupMaster1 = ServerName.valueOf("localhost", 2222, -1);
+    ServerName backupMaster2 = ServerName.valueOf("localhost", 3333, -1);
+    String backupZNode1 = ZKUtil.joinZNode(zk.backupMasterAddressesZNode, backupMaster1.toString());
+    String backupZNode2 = ZKUtil.joinZNode(zk.backupMasterAddressesZNode, backupMaster2.toString());
+    // Add a backup master
+    MasterAddressTracker.setMasterAddress(zk, backupZNode1, backupMaster1, 2222);
+    MasterAddressTracker.setMasterAddress(zk, backupZNode2, backupMaster2, 3333);
+    backupMasters = MasterAddressTracker.getBackupMastersAndRenewWatch(zk);
+    assertEquals(2, backupMasters.size());
+    assertTrue(backupMasters.contains(backupMaster1));
+    assertTrue(backupMasters.contains(backupMaster2));
+  }
+
   public static class NodeCreationListener extends ZooKeeperListener {
     private static final Log LOG = LogFactory.getLog(NodeCreationListener.class);