You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2020/01/29 00:53:26 UTC

[hbase] 02/09: HBASE-23281: Track meta region locations in masters (#830)

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

stack pushed a commit to branch HBASE-18095/client-locate-meta-no-zookeeper
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 791abad2b45916bb604a0f43c77aa449c392beb1
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Wed Dec 4 15:26:58 2019 -0800

    HBASE-23281: Track meta region locations in masters (#830)
    
    * HBASE-23281: Track meta region changes on masters
    
    This patch adds a simple cache that tracks the meta region replica
    locations. It keeps an eye on the region movements so that the
    cached locations are not stale.
    
    This information is used for servicing client RPCs for connections
    that use master based registry (HBASE-18095). The RPC end points
    will be added in a separate patch.
    
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
---
 .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java |  42 +++-
 .../apache/hadoop/hbase/zookeeper/ZNodePaths.java  |  19 +-
 .../org/apache/hadoop/hbase/master/HMaster.java    |  18 +-
 .../hbase/master/MetaRegionLocationCache.java      | 249 +++++++++++++++++++++
 .../hbase/client/TestMetaRegionLocationCache.java  | 186 +++++++++++++++
 .../hbase/master/TestCloseAnOpeningRegion.java     |   5 +-
 .../hbase/master/TestClusterRestartFailover.java   |   2 +-
 .../master/TestRegionsRecoveryConfigManager.java   |   5 +-
 .../hbase/master/TestShutdownBackupMaster.java     |   3 +-
 .../assignment/TestOpenRegionProcedureBackoff.java |   3 +-
 .../assignment/TestOpenRegionProcedureHang.java    |   2 +-
 .../TestRegionAssignedToMultipleRegionServers.java |   3 +-
 .../assignment/TestReportOnlineRegionsRace.java    |   3 +-
 ...tReportRegionStateTransitionFromDeadServer.java |   3 +-
 .../TestReportRegionStateTransitionRetry.java      |   3 +-
 .../master/assignment/TestSCPGetRegionsRace.java   |   3 +-
 .../assignment/TestWakeUpUnexpectedProcedure.java  |   3 +-
 .../TestRegisterPeerWorkerWhenRestarting.java      |   3 +-
 .../hadoop/hbase/protobuf/TestProtobufUtil.java    |  36 ++-
 .../TestRegionServerReportForDuty.java             |   2 +-
 .../replication/TestReplicationProcedureRetry.java |   3 +-
 .../hadoop/hbase/zookeeper/MetaTableLocator.java   |  36 +--
 .../apache/hadoop/hbase/zookeeper/ZKWatcher.java   |  37 ++-
 23 files changed, 586 insertions(+), 83 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 5a71917..2adcea9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -80,6 +80,7 @@ import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionLoadStats;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.RegionStatesCount;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
@@ -93,6 +94,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
 import org.apache.hadoop.hbase.protobuf.ProtobufMessageConverter;
 import org.apache.hadoop.hbase.quotas.QuotaScope;
@@ -3068,6 +3070,44 @@ public final class ProtobufUtil {
   }
 
   /**
+   * Get the Meta region state from the passed data bytes. Can handle both old and new style
+   * server names.
+   * @param data protobuf serialized data with meta server name.
+   * @param replicaId replica ID for this region
+   * @return RegionState instance corresponding to the serialized data.
+   * @throws DeserializationException if the data is invalid.
+   */
+  public static RegionState parseMetaRegionStateFrom(final byte[] data, int replicaId)
+      throws DeserializationException {
+    RegionState.State state = RegionState.State.OPEN;
+    ServerName serverName;
+    if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
+      try {
+        int prefixLen = ProtobufUtil.lengthOfPBMagic();
+        ZooKeeperProtos.MetaRegionServer rl =
+            ZooKeeperProtos.MetaRegionServer.parser().parseFrom(data, prefixLen,
+                data.length - prefixLen);
+        if (rl.hasState()) {
+          state = RegionState.State.convert(rl.getState());
+        }
+        HBaseProtos.ServerName sn = rl.getServer();
+        serverName = ServerName.valueOf(
+            sn.getHostName(), sn.getPort(), sn.getStartCode());
+      } catch (InvalidProtocolBufferException e) {
+        throw new DeserializationException("Unable to parse meta region location");
+      }
+    } else {
+      // old style of meta region location?
+      serverName = parseServerNameFrom(data);
+    }
+    if (serverName == null) {
+      state = RegionState.State.OFFLINE;
+    }
+    return new RegionState(RegionReplicaUtil.getRegionInfoForReplica(
+        RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), state, serverName);
+  }
+
+  /**
    * Get a ServerName from the passed in data bytes.
    * @param data Data with a serialize server name in it; can handle the old style
    * servername where servername was host and port.  Works too with data that
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
index 98d7eb7..5c49808 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
@@ -41,7 +41,8 @@ public class ZNodePaths {
   // TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
   public static final char ZNODE_PATH_SEPARATOR = '/';
 
-  private static final String META_ZNODE_PREFIX = "meta-region-server";
+  public static final String META_ZNODE_PREFIX_CONF_KEY = "zookeeper.znode.metaserver";
+  public static final String META_ZNODE_PREFIX = "meta-region-server";
   private static final String DEFAULT_SNAPSHOT_CLEANUP_ZNODE = "snapshot-cleanup";
 
   // base znode for this cluster
@@ -104,7 +105,7 @@ public class ZNodePaths {
   public ZNodePaths(Configuration conf) {
     baseZNode = conf.get(ZOOKEEPER_ZNODE_PARENT, DEFAULT_ZOOKEEPER_ZNODE_PARENT);
     ImmutableMap.Builder<Integer, String> builder = ImmutableMap.builder();
-    metaZNodePrefix = conf.get("zookeeper.znode.metaserver", META_ZNODE_PREFIX);
+    metaZNodePrefix = conf.get(META_ZNODE_PREFIX_CONF_KEY, META_ZNODE_PREFIX);
     String defaultMetaReplicaZNode = ZNodePaths.joinZNode(baseZNode, metaZNodePrefix);
     builder.put(DEFAULT_REPLICA_ID, defaultMetaReplicaZNode);
     int numMetaReplicas = conf.getInt(META_REPLICAS_NUM, DEFAULT_META_REPLICA_NUM);
@@ -189,7 +190,19 @@ public class ZNodePaths {
   }
 
   /**
-   * Parse the meta replicaId from the passed znode name.
+   * Parses the meta replicaId from the passed path.
+   * @param path the name of the full path which includes baseZNode.
+   * @return replicaId
+   */
+  public int getMetaReplicaIdFromPath(String path) {
+    // Extract the znode from path. The prefix is of the following format.
+    // baseZNode + PATH_SEPARATOR.
+    int prefixLen = baseZNode.length() + 1;
+    return getMetaReplicaIdFromZnode(path.substring(prefixLen));
+  }
+
+  /**
+   * Parse the meta replicaId from the passed znode
    * @param znode the name of the znode, does not include baseZNode
    * @return replicaId
    */
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 3e6f44e..d8f71a9 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
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -355,6 +355,12 @@ public class HMaster extends HRegionServer implements MasterServices {
   // manager of assignment nodes in zookeeper
   private AssignmentManager assignmentManager;
 
+  /**
+   * Cache for the meta region replica's locations. Also tracks their changes to avoid stale
+   * cache entries.
+   */
+  private final MetaRegionLocationCache metaRegionLocationCache;
+
   // manager of replication
   private ReplicationPeerManager replicationPeerManager;
 
@@ -508,8 +514,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    * #finishActiveMasterInitialization(MonitoredTask) after
    * the master becomes the active one.
    */
-  public HMaster(final Configuration conf)
-      throws IOException, KeeperException {
+  public HMaster(final Configuration conf) throws IOException {
     super(conf);
     TraceUtil.initTracer(conf);
     try {
@@ -522,7 +527,6 @@ public class HMaster extends HRegionServer implements MasterServices {
       } else {
         maintenanceMode = false;
       }
-
       this.rsFatals = new MemoryBoundedLogMessageBuffer(
           conf.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024));
       LOG.info("hbase.rootdir={}, hbase.cluster.distributed={}", getDataRootDir(),
@@ -570,8 +574,10 @@ public class HMaster extends HRegionServer implements MasterServices {
 
       // Some unit tests don't need a cluster, so no zookeeper at all
       if (!conf.getBoolean("hbase.testing.nocluster", false)) {
+        this.metaRegionLocationCache = new MetaRegionLocationCache(this.zooKeeper);
         this.activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName, this);
       } else {
+        this.metaRegionLocationCache = null;
         this.activeMasterManager = null;
       }
       cachedClusterId = new CachedClusterId(conf);
@@ -3840,4 +3846,8 @@ public class HMaster extends HRegionServer implements MasterServices {
       rbc.chore();
     }
   }
+
+  public MetaRegionLocationCache getMetaRegionLocationCache() {
+    return this.metaRegionLocationCache;
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
new file mode 100644
index 0000000..f4e91b5
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
@@ -0,0 +1,249 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ThreadFactory;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.types.CopyOnWriteArrayMap;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.hadoop.hbase.util.RetryCounterFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
+/**
+ * A cache of meta region location metadata. Registers a listener on ZK to track changes to the
+ * meta table znodes. Clients are expected to retry if the meta information is stale. This class
+ * is thread-safe (a single instance of this class can be shared by multiple threads without race
+ * conditions).
+ */
+@InterfaceAudience.Private
+public class MetaRegionLocationCache extends ZKListener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MetaRegionLocationCache.class);
+
+  /**
+   * Maximum number of times we retry when ZK operation times out.
+   */
+  private static final int MAX_ZK_META_FETCH_RETRIES = 10;
+  /**
+   * Sleep interval ms between ZK operation retries.
+   */
+  private static final int SLEEP_INTERVAL_MS_BETWEEN_RETRIES = 1000;
+  private static final int SLEEP_INTERVAL_MS_MAX = 10000;
+  private final RetryCounterFactory retryCounterFactory =
+      new RetryCounterFactory(MAX_ZK_META_FETCH_RETRIES, SLEEP_INTERVAL_MS_BETWEEN_RETRIES);
+
+  /**
+   * Cached meta region locations indexed by replica ID.
+   * CopyOnWriteArrayMap ensures synchronization during updates and a consistent snapshot during
+   * client requests. Even though CopyOnWriteArrayMap copies the data structure for every write,
+   * that should be OK since the size of the list is often small and mutations are not too often
+   * and we do not need to block client requests while mutations are in progress.
+   */
+  private final CopyOnWriteArrayMap<Integer, HRegionLocation> cachedMetaLocations;
+
+  private enum ZNodeOpType {
+    INIT,
+    CREATED,
+    CHANGED,
+    DELETED
+  }
+
+  public MetaRegionLocationCache(ZKWatcher zkWatcher) {
+    super(zkWatcher);
+    cachedMetaLocations = new CopyOnWriteArrayMap<>();
+    watcher.registerListener(this);
+    // Populate the initial snapshot of data from meta znodes.
+    // This is needed because stand-by masters can potentially start after the initial znode
+    // creation. It blocks forever until the initial meta locations are loaded from ZK and watchers
+    // are established. Subsequent updates are handled by the registered listener. Also, this runs
+    // in a separate thread in the background to not block master init.
+    ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true).build();
+    RetryCounterFactory retryFactory = new RetryCounterFactory(
+        Integer.MAX_VALUE, SLEEP_INTERVAL_MS_BETWEEN_RETRIES, SLEEP_INTERVAL_MS_MAX);
+    threadFactory.newThread(
+      ()->loadMetaLocationsFromZk(retryFactory.create(), ZNodeOpType.INIT)).start();
+  }
+
+  /**
+   * Populates the current snapshot of meta locations from ZK. If no meta znodes exist, it registers
+   * a watcher on base znode to check for any CREATE/DELETE events on the children.
+   * @param retryCounter controls the number of retries and sleep between retries.
+   */
+  private void loadMetaLocationsFromZk(RetryCounter retryCounter, ZNodeOpType opType) {
+    List<String> znodes = null;
+    while (retryCounter.shouldRetry()) {
+      try {
+        znodes = watcher.getMetaReplicaNodesAndWatchChildren();
+        break;
+      } catch (KeeperException ke) {
+        LOG.debug("Error populating initial meta locations", ke);
+        if (!retryCounter.shouldRetry()) {
+          // Retries exhausted and watchers not set. This is not a desirable state since the cache
+          // could remain stale forever. Propagate the exception.
+          watcher.abort("Error populating meta locations", ke);
+          return;
+        }
+        try {
+          retryCounter.sleepUntilNextRetry();
+        } catch (InterruptedException ie) {
+          LOG.error("Interrupted while loading meta locations from ZK", ie);
+          Thread.currentThread().interrupt();
+          return;
+        }
+      }
+    }
+    if (znodes == null || znodes.isEmpty()) {
+      // No meta znodes exist at this point but we registered a watcher on the base znode to listen
+      // for updates. They will be handled via nodeChildrenChanged().
+      return;
+    }
+    if (znodes.size() == cachedMetaLocations.size()) {
+      // No new meta znodes got added.
+      return;
+    }
+    for (String znode: znodes) {
+      String path = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, znode);
+      updateMetaLocation(path, opType);
+    }
+  }
+
+  /**
+   * Gets the HRegionLocation for a given meta replica ID. Renews the watch on the znode for
+   * future updates.
+   * @param replicaId ReplicaID of the region.
+   * @return HRegionLocation for the meta replica.
+   * @throws KeeperException if there is any issue fetching/parsing the serialized data.
+   */
+  private HRegionLocation getMetaRegionLocation(int replicaId)
+      throws KeeperException {
+    RegionState metaRegionState;
+    try {
+      byte[] data = ZKUtil.getDataAndWatch(watcher,
+          watcher.getZNodePaths().getZNodeForReplica(replicaId));
+      metaRegionState = ProtobufUtil.parseMetaRegionStateFrom(data, replicaId);
+    } catch (DeserializationException e) {
+      throw ZKUtil.convert(e);
+    }
+    return new HRegionLocation(metaRegionState.getRegion(), metaRegionState.getServerName());
+  }
+
+  private void updateMetaLocation(String path, ZNodeOpType opType) {
+    if (!isValidMetaZNode(path)) {
+      return;
+    }
+    LOG.debug("Updating meta znode for path {}: {}", path, opType.name());
+    int replicaId = watcher.getZNodePaths().getMetaReplicaIdFromPath(path);
+    RetryCounter retryCounter = retryCounterFactory.create();
+    HRegionLocation location = null;
+    while (retryCounter.shouldRetry()) {
+      try {
+        if (opType == ZNodeOpType.DELETED) {
+          if (!ZKUtil.watchAndCheckExists(watcher, path)) {
+            // The path does not exist, we've set the watcher and we can break for now.
+            break;
+          }
+          // If it is a transient error and the node appears right away, we fetch the
+          // latest meta state.
+        }
+        location = getMetaRegionLocation(replicaId);
+        break;
+      } catch (KeeperException e) {
+        LOG.debug("Error getting meta location for path {}", path, e);
+        if (!retryCounter.shouldRetry()) {
+          LOG.warn("Error getting meta location for path {}. Retries exhausted.", path, e);
+          break;
+        }
+        try {
+          retryCounter.sleepUntilNextRetry();
+        } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
+          return;
+        }
+      }
+    }
+    if (location == null) {
+      cachedMetaLocations.remove(replicaId);
+      return;
+    }
+    cachedMetaLocations.put(replicaId, location);
+  }
+
+  /**
+   * @return Optional list of HRegionLocations for meta replica(s), null if the cache is empty.
+   *
+   */
+  public Optional<List<HRegionLocation>> getMetaRegionLocations() {
+    ConcurrentNavigableMap<Integer, HRegionLocation> snapshot =
+        cachedMetaLocations.tailMap(cachedMetaLocations.firstKey());
+    if (snapshot.isEmpty()) {
+      // This could be possible if the master has not successfully initialized yet or meta region
+      // is stuck in some weird state.
+      return Optional.empty();
+    }
+    List<HRegionLocation> result = new ArrayList<>();
+    // Explicitly iterate instead of new ArrayList<>(snapshot.values()) because the underlying
+    // ArrayValueCollection does not implement toArray().
+    snapshot.values().forEach(location -> result.add(location));
+    return Optional.of(result);
+  }
+
+  /**
+   * Helper to check if the given 'path' corresponds to a meta znode. This listener is only
+   * interested in changes to meta znodes.
+   */
+  private boolean isValidMetaZNode(String path) {
+    return watcher.getZNodePaths().isAnyMetaReplicaZNode(path);
+  }
+
+  @Override
+  public void nodeCreated(String path) {
+    updateMetaLocation(path, ZNodeOpType.CREATED);
+  }
+
+  @Override
+  public void nodeDeleted(String path) {
+    updateMetaLocation(path, ZNodeOpType.DELETED);
+  }
+
+  @Override
+  public void nodeDataChanged(String path) {
+    updateMetaLocation(path, ZNodeOpType.CHANGED);
+  }
+
+  @Override
+  public void nodeChildrenChanged(String path) {
+    if (!path.equals(watcher.getZNodePaths().baseZNode)) {
+      return;
+    }
+    loadMetaLocationsFromZk(retryCounterFactory.create(), ZNodeOpType.CHANGED);
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
new file mode 100644
index 0000000..02236a6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MultithreadedTestUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MetaRegionLocationCache;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({SmallTests.class, MasterTests.class })
+public class TestMetaRegionLocationCache {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMetaRegionLocationCache.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static AsyncRegistry REGISTRY;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
+    TEST_UTIL.startMiniCluster(3);
+    REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(
+        TEST_UTIL.getConfiguration(), REGISTRY, 3);
+    TEST_UTIL.getAdmin().balancerSwitch(false, true);
+  }
+
+  @AfterClass
+  public static void cleanUp() throws Exception {
+    IOUtils.closeQuietly(REGISTRY);
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  private List<HRegionLocation> getCurrentMetaLocations(ZKWatcher zk) throws Exception {
+    List<HRegionLocation> result = new ArrayList<>();
+    for (String znode: zk.getMetaReplicaNodes()) {
+      String path = ZNodePaths.joinZNode(zk.getZNodePaths().baseZNode, znode);
+      int replicaId = zk.getZNodePaths().getMetaReplicaIdFromPath(path);
+      RegionState state = MetaTableLocator.getMetaRegionState(zk, replicaId);
+      result.add(new HRegionLocation(state.getRegion(), state.getServerName()));
+    }
+    return result;
+  }
+
+  // Verifies that the cached meta locations in the given master are in sync with what is in ZK.
+  private void verifyCachedMetaLocations(HMaster master) throws Exception {
+    // Wait until initial meta locations are loaded.
+    int retries = 0;
+    while (!master.getMetaRegionLocationCache().getMetaRegionLocations().isPresent()) {
+      Thread.sleep(1000);
+      if (++retries == 10) {
+        break;
+      }
+    }
+    List<HRegionLocation> metaHRLs =
+        master.getMetaRegionLocationCache().getMetaRegionLocations().get();
+    assertFalse(metaHRLs.isEmpty());
+    ZKWatcher zk = master.getZooKeeper();
+    List<String> metaZnodes = zk.getMetaReplicaNodes();
+    assertEquals(metaZnodes.size(), metaHRLs.size());
+    List<HRegionLocation> actualHRLs = getCurrentMetaLocations(zk);
+    Collections.sort(metaHRLs);
+    Collections.sort(actualHRLs);
+    assertEquals(actualHRLs, metaHRLs);
+  }
+
+  @Test public void testInitialMetaLocations() throws Exception {
+    verifyCachedMetaLocations(TEST_UTIL.getMiniHBaseCluster().getMaster());
+  }
+
+  @Test public void testStandByMetaLocations() throws Exception {
+    HMaster standBy = TEST_UTIL.getMiniHBaseCluster().startMaster().getMaster();
+    verifyCachedMetaLocations(standBy);
+  }
+
+  /*
+   * Shuffles the meta region replicas around the cluster and makes sure the cache is not stale.
+   */
+  @Test public void testMetaLocationsChange() throws Exception {
+    List<HRegionLocation> currentMetaLocs =
+        getCurrentMetaLocations(TEST_UTIL.getMiniHBaseCluster().getMaster().getZooKeeper());
+    // Move these replicas to random servers.
+    for (HRegionLocation location: currentMetaLocs) {
+      RegionReplicaTestHelper.moveRegion(TEST_UTIL, location);
+    }
+    RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(
+        TEST_UTIL.getConfiguration(), REGISTRY, 3);
+    for (JVMClusterUtil.MasterThread masterThread:
+        TEST_UTIL.getMiniHBaseCluster().getMasterThreads()) {
+      verifyCachedMetaLocations(masterThread.getMaster());
+    }
+  }
+
+  /**
+   * Tests MetaRegionLocationCache's init procedure to make sure that it correctly watches the base
+   * znode for notifications.
+   */
+  @Test public void testMetaRegionLocationCache() throws Exception {
+    final String parentZnodeName = "/randomznodename";
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parentZnodeName);
+    ServerName sn = ServerName.valueOf("localhost", 1234, 5678);
+    try (ZKWatcher zkWatcher = new ZKWatcher(conf, null, null, true)) {
+      // A thread that repeatedly creates and drops an unrelated child znode. This is to simulate
+      // some ZK activity in the background.
+      MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(conf);
+      ctx.addThread(new MultithreadedTestUtil.RepeatingTestThread(ctx) {
+        @Override public void doAnAction() throws Exception {
+          final String testZnode = parentZnodeName + "/child";
+          ZKUtil.createNodeIfNotExistsAndWatch(zkWatcher, testZnode, testZnode.getBytes());
+          ZKUtil.deleteNode(zkWatcher, testZnode);
+        }
+      });
+      ctx.startThreads();
+      try {
+        MetaRegionLocationCache metaCache = new MetaRegionLocationCache(zkWatcher);
+        // meta znodes do not exist at this point, cache should be empty.
+        assertFalse(metaCache.getMetaRegionLocations().isPresent());
+        // Set the meta locations for a random meta replicas, simulating an active hmaster meta
+        // assignment.
+        for (int i = 0; i < 3; i++) {
+          // Updates the meta znodes.
+          MetaTableLocator.setMetaLocation(zkWatcher, sn, i, RegionState.State.OPEN);
+        }
+        // Wait until the meta cache is populated.
+        int iters = 0;
+        while (iters++ < 10) {
+          if (metaCache.getMetaRegionLocations().isPresent()
+            && metaCache.getMetaRegionLocations().get().size() == 3) {
+            break;
+          }
+          Thread.sleep(1000);
+        }
+        List<HRegionLocation> metaLocations = metaCache.getMetaRegionLocations().get();
+        assertEquals(3, metaLocations.size());
+        for (HRegionLocation location : metaLocations) {
+          assertEquals(sn, location.getServerName());
+        }
+      } finally {
+        // clean up.
+        ctx.stop();
+        ZKUtil.deleteChildrenRecursively(zkWatcher, parentZnodeName);
+      }
+    }
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java
index ba4d535..4922224 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -65,7 +64,7 @@ public class TestCloseAnOpeningRegion {
 
   public static final class MockHMaster extends HMaster {
 
-    public MockHMaster(Configuration conf) throws IOException, KeeperException {
+    public MockHMaster(Configuration conf) throws IOException {
       super(conf);
     }
 
@@ -141,4 +140,4 @@ public class TestCloseAnOpeningRegion {
       table.put(new Put(Bytes.toBytes(0)).addColumn(CF, Bytes.toBytes("cq"), Bytes.toBytes(0)));
     }
   }
-}
\ No newline at end of file
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java
index 338173e..1cce2ac 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java
@@ -164,7 +164,7 @@ public class TestClusterRestartFailover extends AbstractTestRestartCluster {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java
index 22554d3..d29e061 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -120,7 +119,7 @@ public class TestRegionsRecoveryConfigManager {
 
   // Make it public so that JVMClusterUtil can access it.
   public static class TestHMaster extends HMaster {
-    public TestHMaster(Configuration conf) throws IOException, KeeperException {
+    public TestHMaster(Configuration conf) throws IOException {
       super(conf);
     }
   }
@@ -144,4 +143,4 @@ public class TestRegionsRecoveryConfigManager {
 
   }
 
-}
\ No newline at end of file
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java
index d3a8520..a42a404 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -56,7 +55,7 @@ public class TestShutdownBackupMaster {
 
   public static final class MockHMaster extends HMaster {
 
-    public MockHMaster(Configuration conf) throws IOException, KeeperException {
+    public MockHMaster(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java
index ca0384e..4112da7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -71,7 +70,7 @@ public class TestOpenRegionProcedureBackoff {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java
index 0463721..a25368f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java
@@ -105,7 +105,7 @@ public class TestOpenRegionProcedureHang {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java
index 0d8202b..44af256 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -110,7 +109,7 @@ public class TestRegionAssignedToMultipleRegionServers {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java
index acad88c..4dede89 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.IdLock;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -106,7 +105,7 @@ public class TestReportOnlineRegionsRace {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java
index 6c9e5eb..1de806f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -117,7 +116,7 @@ public class TestReportRegionStateTransitionFromDeadServer {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java
index 6c191c9..71c4693 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -84,7 +83,7 @@ public class TestReportRegionStateTransitionRetry {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java
index cbbdbdc..d676af9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.IdLock;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -130,7 +129,7 @@ public class TestSCPGetRegionsRace {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java
index 47c70a1..62e3161 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -198,7 +197,7 @@ public class TestWakeUpUnexpectedProcedure {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java
index f46bb41..4dff86d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.replication.SyncReplicationTestBase;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
-import org.apache.zookeeper.KeeperException;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -53,7 +52,7 @@ public class TestRegisterPeerWorkerWhenRestarting extends SyncReplicationTestBas
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
index ff29df8..69e656f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.protobuf;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-
 import com.google.protobuf.ByteString;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -29,13 +28,17 @@ import org.apache.hadoop.hbase.CellBuilderType;
 import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
@@ -51,11 +54,12 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer;
 
 /**
  * Class to test ProtobufUtil.
  */
-@Category({MiscTests.class, SmallTests.class})
+@Category({ MiscTests.class, SmallTests.class})
 public class TestProtobufUtil {
 
   @ClassRule
@@ -348,4 +352,32 @@ public class TestProtobufUtil {
         ProtobufUtil.toCell(ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), cell);
     assertTrue(CellComparatorImpl.COMPARATOR.compare(offheapKV, newOffheapKV) == 0);
   }
+
+  @Test
+  public void testMetaRegionState() throws Exception {
+    ServerName serverName = ServerName.valueOf("localhost", 1234, 5678);
+    // New region state style.
+    for (RegionState.State state: RegionState.State.values()) {
+      RegionState regionState =
+          new RegionState(RegionInfoBuilder.FIRST_META_REGIONINFO, state, serverName);
+      MetaRegionServer metars = MetaRegionServer.newBuilder()
+          .setServer(org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toServerName(serverName))
+          .setRpcVersion(HConstants.RPC_CURRENT_VERSION)
+          .setState(state.convert()).build();
+      // Serialize
+      byte[] data = ProtobufUtil.prependPBMagic(metars.toByteArray());
+      ProtobufUtil.prependPBMagic(data);
+      // Deserialize
+      RegionState regionStateNew =
+          org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseMetaRegionStateFrom(data, 1);
+      assertEquals(regionState.getServerName(), regionStateNew.getServerName());
+      assertEquals(regionState.getState(), regionStateNew.getState());
+    }
+    // old style.
+    RegionState rs =
+        org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseMetaRegionStateFrom(
+            serverName.getVersionedBytes(), 1);
+    assertEquals(serverName, rs.getServerName());
+    assertEquals(rs.getState(), RegionState.State.OPEN);
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java
index aaf2d2e..f61a77e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java
@@ -122,7 +122,7 @@ public class TestRegionServerReportForDuty {
    * This test HMaster class will always throw ServerNotRunningYetException if checked.
    */
   public static class NeverInitializedMaster extends HMaster {
-    public NeverInitializedMaster(Configuration conf) throws IOException, KeeperException {
+    public NeverInitializedMaster(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
index a2ae0b4..e9fcc66 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -134,7 +133,7 @@ public class TestReplicationProcedureRetry {
 
     private ReplicationPeerManager manager;
 
-    public MockHMaster(Configuration conf) throws IOException, KeeperException {
+    public MockHMaster(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
index 9105d7e..c7b45fe 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
@@ -34,12 +34,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
-
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer;
 
 /**
@@ -266,40 +261,17 @@ public final class MetaTableLocator {
    * @throws KeeperException if a ZooKeeper operation fails
    */
   public static RegionState getMetaRegionState(ZKWatcher zkw, int replicaId)
-          throws KeeperException {
-    RegionState.State state = RegionState.State.OPEN;
-    ServerName serverName = null;
+      throws KeeperException {
+    RegionState regionState = null;
     try {
       byte[] data = ZKUtil.getData(zkw, zkw.getZNodePaths().getZNodeForReplica(replicaId));
-      if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
-        try {
-          int prefixLen = ProtobufUtil.lengthOfPBMagic();
-          ZooKeeperProtos.MetaRegionServer rl =
-            ZooKeeperProtos.MetaRegionServer.parser().parseFrom(data, prefixLen,
-                    data.length - prefixLen);
-          if (rl.hasState()) {
-            state = RegionState.State.convert(rl.getState());
-          }
-          HBaseProtos.ServerName sn = rl.getServer();
-          serverName = ServerName.valueOf(
-            sn.getHostName(), sn.getPort(), sn.getStartCode());
-        } catch (InvalidProtocolBufferException e) {
-          throw new DeserializationException("Unable to parse meta region location");
-        }
-      } else {
-        // old style of meta region location?
-        serverName = ProtobufUtil.parseServerNameFrom(data);
-      }
+      regionState = ProtobufUtil.parseMetaRegionStateFrom(data, replicaId);
     } catch (DeserializationException e) {
       throw ZKUtil.convert(e);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
     }
-    if (serverName == null) {
-      state = RegionState.State.OFFLINE;
-    }
-    return new RegionState(RegionReplicaUtil.getRegionInfoForReplica(
-        RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), state, serverName);
+    return regionState;
   }
 
   /**
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
index e0b34e4..a39c413 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
@@ -23,10 +23,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.CountDownLatch;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.AuthUtil;
@@ -81,10 +79,6 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
   // listeners to be notified
   private final List<ZKListener> listeners = new CopyOnWriteArrayList<>();
 
-  // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL
-  // negotiation to complete
-  private CountDownLatch saslLatch = new CountDownLatch(1);
-
   private final Configuration conf;
 
   /* A pattern that matches a Kerberos name, borrowed from Hadoop's KerberosName */
@@ -383,13 +377,32 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
    */
   public List<String> getMetaReplicaNodes() throws KeeperException {
     List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode);
+    return filterMetaReplicaNodes(childrenOfBaseNode);
+  }
+
+  /**
+   * Same as {@link #getMetaReplicaNodes()} except that this also registers a watcher on base znode
+   * for subsequent CREATE/DELETE operations on child nodes.
+   */
+  public List<String> getMetaReplicaNodesAndWatchChildren() throws KeeperException {
+    List<String> childrenOfBaseNode =
+        ZKUtil.listChildrenAndWatchForNewChildren(this, znodePaths.baseZNode);
+    return filterMetaReplicaNodes(childrenOfBaseNode);
+  }
+
+  /**
+   * @param nodes Input list of znodes
+   * @return Filtered list of znodes from nodes that belong to meta replica(s).
+   */
+  private List<String> filterMetaReplicaNodes(List<String> nodes) {
+    if (nodes == null || nodes.isEmpty()) {
+      return new ArrayList<>();
+    }
     List<String> metaReplicaNodes = new ArrayList<>(2);
-    if (childrenOfBaseNode != null) {
-      String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server");
-      for (String child : childrenOfBaseNode) {
-        if (child.startsWith(pattern)) {
-          metaReplicaNodes.add(child);
-        }
+    String pattern = conf.get(ZNodePaths.META_ZNODE_PREFIX_CONF_KEY, ZNodePaths.META_ZNODE_PREFIX);
+    for (String child : nodes) {
+      if (child.startsWith(pattern)) {
+        metaReplicaNodes.add(child);
       }
     }
     return metaReplicaNodes;