You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2018/12/04 14:33:58 UTC

[2/2] hbase git commit: HBASE-21541 Move MetaTableLocator.verifyRegionLocation to hbase-rsgroup module

HBASE-21541 Move MetaTableLocator.verifyRegionLocation to hbase-rsgroup module


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

Branch: refs/heads/master
Commit: 8bf966c8e936dec4d83bcbe85c5aab543f14a0df
Parents: 59cfe2e
Author: zhangduo <zh...@apache.org>
Authored: Tue Dec 4 22:33:36 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 4 22:33:36 2018 +0800

----------------------------------------------------------------------
 .../hbase/rsgroup/RSGroupInfoManagerImpl.java   |   3 +-
 .../apache/hadoop/hbase/rsgroup/Utility.java    | 196 ++++++++++-
 .../hadoop/hbase/rsgroup/TestUtility.java       | 229 +++++++++++++
 .../java/org/apache/hadoop/hbase/Server.java    |  15 +-
 .../hbase/master/MasterMetaBootstrap.java       |   3 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   5 +-
 .../hbase/master/MasterStatusServlet.java       |   8 +-
 .../master/procedure/ProcedureSyncWait.java     |   8 +-
 .../master/snapshot/MasterSnapshotVerifier.java |   4 +-
 .../master/snapshot/TakeSnapshotHandler.java    |   4 +-
 .../flush/MasterFlushTableProcedureManager.java |   6 +-
 .../hbase/regionserver/HRegionServer.java       |  16 -
 .../regionserver/ReplicationSyncUp.java         |   6 -
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |  10 +-
 .../resources/hbase-webapps/master/table.jsp    |   4 +-
 .../hadoop/hbase/MockRegionServerServices.java  |   6 -
 .../hadoop/hbase/TestMetaTableAccessor.java     |  10 +-
 .../hadoop/hbase/TestMetaTableLocator.java      | 261 +++------------
 .../hbase/client/TestMetaWithReplicas.java      |   4 +-
 .../hbase/master/MockNoopMasterServices.java    |   6 -
 .../hadoop/hbase/master/MockRegionServer.java   |   6 -
 .../hbase/master/TestActiveMasterManager.java   |   6 -
 .../hbase/master/cleaner/TestHFileCleaner.java  |   6 -
 .../master/cleaner/TestHFileLinkCleaner.java    |   6 -
 .../hbase/master/cleaner/TestLogsCleaner.java   |   7 -
 .../cleaner/TestReplicationHFileCleaner.java    |   6 -
 .../regionserver/TestHeapMemoryManager.java     |   6 -
 .../regionserver/TestRegionServerNoMaster.java  |   5 +-
 .../hbase/regionserver/TestSplitLogWorker.java  |   6 -
 .../hbase/regionserver/TestWALLockup.java       |   6 -
 .../TestReplicationTrackerZKImpl.java           |   6 -
 .../TestReplicationSourceManager.java           |   6 -
 .../security/token/TestTokenAuthentication.java |   6 -
 .../apache/hadoop/hbase/util/MockServer.java    |   7 -
 .../hbase/zookeeper/MetaTableLocator.java       | 329 +++----------------
 .../apache/hadoop/hbase/zookeeper/ZKUtil.java   |   7 +-
 36 files changed, 551 insertions(+), 674 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index cdd68d1..6f54c31 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -757,8 +757,7 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
         found.set(true);
         try {
           boolean rootMetaFound =
-              masterServices.getMetaTableLocator().verifyMetaRegionLocation(
-                  conn, masterServices.getZooKeeper(), 1);
+            Utility.verifyMetaRegionLocation(conn, masterServices.getZooKeeper(), 1);
           if (rootMetaFound) {
             MetaTableAccessor.Visitor visitor = new DefaultVisitorBase() {
               @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/Utility.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/Utility.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/Utility.java
index 4d7bf43..d5450c4 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/Utility.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/Utility.java
@@ -19,19 +19,49 @@
  */
 package org.apache.hadoop.hbase.rsgroup;
 
+import java.io.EOFException;
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.NoRouteToHostException;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
 import java.util.HashSet;
+import java.util.Locale;
 import java.util.Set;
-
+import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.RetriesExhaustedException;
+import org.apache.hadoop.hbase.ipc.FailedServerException;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.net.Address;
+import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 
 /**
  * Utility for this RSGroup package in hbase-rsgroup.
  */
 @InterfaceAudience.Private
 final class Utility {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Utility.class);
+
   private Utility() {
   }
 
@@ -45,9 +75,171 @@ final class Utility {
       return onlineServers;
     }
 
-    for(ServerName server: master.getServerManager().getOnlineServers().keySet()) {
+    for (ServerName server : master.getServerManager().getOnlineServers().keySet()) {
       onlineServers.add(server.getAddress());
     }
     return onlineServers;
   }
+
+  /**
+   * Verify <code>hbase:meta</code> is deployed and accessible.
+   * @param hConnection the connection to use
+   * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
+   * @param timeout How long to wait on zk for meta address (passed through to the internal call to
+   *          {@link #getMetaServerConnection}.
+   * @return True if the <code>hbase:meta</code> location is healthy.
+   * @throws IOException if the number of retries for getting the connection is exceeded
+   * @throws InterruptedException if waiting for the socket operation fails
+   */
+  public static boolean verifyMetaRegionLocation(ClusterConnection hConnection, ZKWatcher zkw,
+      final long timeout) throws InterruptedException, IOException {
+    return verifyMetaRegionLocation(hConnection, zkw, timeout, RegionInfo.DEFAULT_REPLICA_ID);
+  }
+
+  /**
+   * Verify <code>hbase:meta</code> is deployed and accessible.
+   * @param connection the connection to use
+   * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
+   * @param timeout How long to wait on zk for meta address (passed through to
+   * @param replicaId the ID of the replica
+   * @return True if the <code>hbase:meta</code> location is healthy.
+   * @throws InterruptedException if waiting for the socket operation fails
+   * @throws IOException if the number of retries for getting the connection is exceeded
+   */
+  public static boolean verifyMetaRegionLocation(ClusterConnection connection, ZKWatcher zkw,
+      final long timeout, int replicaId) throws InterruptedException, IOException {
+    AdminProtos.AdminService.BlockingInterface service = null;
+    try {
+      service = getMetaServerConnection(connection, zkw, timeout, replicaId);
+    } catch (NotAllMetaRegionsOnlineException e) {
+      // Pass
+    } catch (ServerNotRunningYetException e) {
+      // Pass -- remote server is not up so can't be carrying root
+    } catch (UnknownHostException e) {
+      // Pass -- server name doesn't resolve so it can't be assigned anything.
+    } catch (RegionServerStoppedException e) {
+      // Pass -- server name sends us to a server that is dying or already dead.
+    }
+    return (service != null) && verifyRegionLocation(connection, service,
+      MetaTableLocator.getMetaRegionLocation(zkw, replicaId),
+      RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId)
+        .getRegionName());
+  }
+
+  /**
+   * Verify we can connect to <code>hostingServer</code> and that its carrying
+   * <code>regionName</code>.
+   * @param hostingServer Interface to the server hosting <code>regionName</code>
+   * @param address The servername that goes with the <code>metaServer</code> interface. Used
+   *          logging.
+   * @param regionName The regionname we are interested in.
+   * @return True if we were able to verify the region located at other side of the interface.
+   */
+  // TODO: We should be able to get the ServerName from the AdminProtocol
+  // rather than have to pass it in. Its made awkward by the fact that the
+  // HRI is likely a proxy against remote server so the getServerName needs
+  // to be fixed to go to a local method or to a cache before we can do this.
+  private static boolean verifyRegionLocation(final ClusterConnection connection,
+      AdminService.BlockingInterface hostingServer, final ServerName address,
+      final byte[] regionName) {
+    if (hostingServer == null) {
+      LOG.info("Passed hostingServer is null");
+      return false;
+    }
+    Throwable t;
+    HBaseRpcController controller = connection.getRpcControllerFactory().newController();
+    try {
+      // Try and get regioninfo from the hosting server.
+      return ProtobufUtil.getRegionInfo(controller, hostingServer, regionName) != null;
+    } catch (ConnectException e) {
+      t = e;
+    } catch (RetriesExhaustedException e) {
+      t = e;
+    } catch (RemoteException e) {
+      IOException ioe = e.unwrapRemoteException();
+      t = ioe;
+    } catch (IOException e) {
+      Throwable cause = e.getCause();
+      if (cause != null && cause instanceof EOFException) {
+        t = cause;
+      } else if (cause != null && cause.getMessage() != null &&
+        cause.getMessage().contains("Connection reset")) {
+        t = cause;
+      } else {
+        t = e;
+      }
+    }
+    LOG.info("Failed verification of " + Bytes.toStringBinary(regionName) + " at address=" +
+      address + ", exception=" + t.getMessage());
+    return false;
+  }
+
+  /**
+   * Gets a connection to the server hosting meta, as reported by ZooKeeper, waiting up to the
+   * specified timeout for availability.
+   * <p>
+   * WARNING: Does not retry. Use an {@link org.apache.hadoop.hbase.client.HTable} instead.
+   * @param connection the connection to use
+   * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
+   * @param timeout How long to wait on meta location
+   * @param replicaId the ID of the replica
+   * @return connection to server hosting meta
+   * @throws InterruptedException if waiting for the socket operation fails
+   * @throws IOException if the number of retries for getting the connection is exceeded
+   */
+  private static AdminService.BlockingInterface getMetaServerConnection(
+      ClusterConnection connection, ZKWatcher zkw, long timeout, int replicaId)
+      throws InterruptedException, IOException {
+    return getCachedConnection(connection,
+      MetaTableLocator.waitMetaRegionLocation(zkw, replicaId, timeout));
+  }
+
+  /**
+   * @param sn ServerName to get a connection against.
+   * @return The AdminProtocol we got when we connected to <code>sn</code> May have come from cache,
+   *         may not be good, may have been setup by this invocation, or may be null.
+   * @throws IOException if the number of retries for getting the connection is exceeded
+   */
+  private static AdminService.BlockingInterface getCachedConnection(ClusterConnection connection,
+      ServerName sn) throws IOException {
+    if (sn == null) {
+      return null;
+    }
+    AdminService.BlockingInterface service = null;
+    try {
+      service = connection.getAdmin(sn);
+    } catch (RetriesExhaustedException e) {
+      if (e.getCause() != null && e.getCause() instanceof ConnectException) {
+        LOG.debug("Catch this; presume it means the cached connection has gone bad.");
+      } else {
+        throw e;
+      }
+    } catch (SocketTimeoutException e) {
+      LOG.debug("Timed out connecting to " + sn);
+    } catch (NoRouteToHostException e) {
+      LOG.debug("Connecting to " + sn, e);
+    } catch (SocketException e) {
+      LOG.debug("Exception connecting to " + sn);
+    } catch (UnknownHostException e) {
+      LOG.debug("Unknown host exception connecting to  " + sn);
+    } catch (FailedServerException e) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Server " + sn + " is in failed server list.");
+      }
+    } catch (IOException ioe) {
+      Throwable cause = ioe.getCause();
+      if (ioe instanceof ConnectException) {
+        LOG.debug("Catch. Connect refused.");
+      } else if (cause != null && cause instanceof EOFException) {
+        LOG.debug("Catch. Other end disconnected us.");
+      } else if (cause != null && cause.getMessage() != null &&
+        cause.getMessage().toLowerCase(Locale.ROOT).contains("connection reset")) {
+        LOG.debug("Catch. Connection reset.");
+      } else {
+        throw ioe;
+      }
+
+    }
+    return service;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestUtility.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestUtility.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestUtility.java
new file mode 100644
index 0000000..0065610
--- /dev/null
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestUtility.java
@@ -0,0 +1,229 @@
+/**
+ * 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.rsgroup;
+
+import static org.junit.Assert.assertFalse;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+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.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;
+
+@Category({ MiscTests.class, MediumTests.class })
+public class TestUtility {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestUtility.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestUtility.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static final ServerName SN =
+    ServerName.valueOf("example.org", 1234, System.currentTimeMillis());
+
+  private ZKWatcher watcher;
+
+  private Abortable abortable;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // Set this down so tests run quicker
+    UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
+    UTIL.startMiniZKCluster();
+  }
+
+  @AfterClass
+  public static void afterClass() throws IOException {
+    UTIL.getZkCluster().shutdown();
+  }
+
+  @Before
+  public void before() throws IOException {
+    this.abortable = new Abortable() {
+      @Override
+      public void abort(String why, Throwable e) {
+        LOG.info(why, e);
+      }
+
+      @Override
+      public boolean isAborted() {
+        return false;
+      }
+    };
+    this.watcher =
+      new ZKWatcher(UTIL.getConfiguration(), this.getClass().getSimpleName(), this.abortable, true);
+  }
+
+  @After
+  public void after() {
+    try {
+      // Clean out meta location or later tests will be confused... they presume
+      // start fresh in zk.
+      MetaTableLocator.deleteMetaLocation(this.watcher);
+    } catch (KeeperException e) {
+      LOG.warn("Unable to delete hbase:meta location", e);
+    }
+
+    this.watcher.close();
+  }
+
+  /**
+   * @param admin An {@link AdminProtos.AdminService.BlockingInterface} instance; you'll likely want
+   *          to pass a mocked HRS; can be null.
+   * @param client A mocked ClientProtocol instance, can be null
+   * @return Mock up a connection that returns a {@link Configuration} when
+   *         {@link org.apache.hadoop.hbase.client.ClusterConnection#getConfiguration()} is called,
+   *         a 'location' when
+   *         {@link org.apache.hadoop.hbase.client.RegionLocator#getRegionLocation(byte[], boolean)}
+   *         is called, and that returns the passed
+   *         {@link AdminProtos.AdminService.BlockingInterface} instance when
+   *         {@link org.apache.hadoop.hbase.client.ClusterConnection#getAdmin(ServerName)} is
+   *         called, returns the passed {@link ClientProtos.ClientService.BlockingInterface}
+   *         instance when
+   *         {@link org.apache.hadoop.hbase.client.ClusterConnection#getClient(ServerName)} is
+   *         called.
+   */
+  private ClusterConnection mockConnection(final AdminProtos.AdminService.BlockingInterface admin,
+      final ClientProtos.ClientService.BlockingInterface client) throws IOException {
+    ClusterConnection connection =
+      HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
+    Mockito.doNothing().when(connection).close();
+    // Make it so we return any old location when asked.
+    final HRegionLocation anyLocation =
+      new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, SN);
+    Mockito.when(connection.getRegionLocation((TableName) Mockito.any(), (byte[]) Mockito.any(),
+      Mockito.anyBoolean())).thenReturn(anyLocation);
+    Mockito.when(connection.locateRegion((TableName) Mockito.any(), (byte[]) Mockito.any()))
+      .thenReturn(anyLocation);
+    if (admin != null) {
+      // If a call to getHRegionConnection, return this implementation.
+      Mockito.when(connection.getAdmin(Mockito.any())).thenReturn(admin);
+    }
+    if (client != null) {
+      // If a call to getClient, return this implementation.
+      Mockito.when(connection.getClient(Mockito.any())).thenReturn(client);
+    }
+    return connection;
+  }
+
+  private void testVerifyMetaRegionLocationWithException(Exception ex)
+      throws IOException, InterruptedException, KeeperException, ServiceException {
+    // Mock an ClientProtocol.
+    final ClientProtos.ClientService.BlockingInterface implementation =
+      Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
+
+    ClusterConnection connection = mockConnection(null, implementation);
+
+    // If a 'get' is called on mocked interface, throw connection refused.
+    Mockito.when(implementation.get((RpcController) Mockito.any(), (GetRequest) Mockito.any()))
+      .thenThrow(new ServiceException(ex));
+
+    long timeout = UTIL.getConfiguration().getLong("hbase.catalog.verification.timeout", 1000);
+    MetaTableLocator.setMetaLocation(this.watcher, SN, RegionState.State.OPENING);
+    assertFalse(Utility.verifyMetaRegionLocation(connection, watcher, timeout));
+
+    MetaTableLocator.setMetaLocation(this.watcher, SN, RegionState.State.OPEN);
+    assertFalse(Utility.verifyMetaRegionLocation(connection, watcher, timeout));
+  }
+
+  /**
+   * Test get of meta region fails properly if nothing to connect to.
+   */
+  @Test
+  public void testVerifyMetaRegionLocationFails()
+      throws IOException, InterruptedException, KeeperException, ServiceException {
+    ClusterConnection connection = Mockito.mock(ClusterConnection.class);
+    ServiceException connectException =
+      new ServiceException(new ConnectException("Connection refused"));
+    final AdminProtos.AdminService.BlockingInterface implementation =
+      Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
+    Mockito.when(implementation.getRegionInfo((RpcController) Mockito.any(),
+      (GetRegionInfoRequest) Mockito.any())).thenThrow(connectException);
+    Mockito.when(connection.getAdmin(Mockito.any())).thenReturn(implementation);
+    RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class);
+    Mockito.when(controllerFactory.newController())
+      .thenReturn(Mockito.mock(HBaseRpcController.class));
+    Mockito.when(connection.getRpcControllerFactory()).thenReturn(controllerFactory);
+
+    ServerName sn = ServerName.valueOf("example.com", 1234, System.currentTimeMillis());
+    MetaTableLocator.setMetaLocation(this.watcher, sn, RegionState.State.OPENING);
+    assertFalse(Utility.verifyMetaRegionLocation(connection, watcher, 100));
+    MetaTableLocator.setMetaLocation(this.watcher, sn, RegionState.State.OPEN);
+    assertFalse(Utility.verifyMetaRegionLocation(connection, watcher, 100));
+  }
+
+  /**
+   * Test we survive a connection refused {@link ConnectException}
+   */
+  @Test
+  public void testGetMetaServerConnectionFails()
+      throws IOException, InterruptedException, KeeperException, ServiceException {
+    testVerifyMetaRegionLocationWithException(new ConnectException("Connection refused"));
+  }
+
+  /**
+   * Test that verifyMetaRegionLocation properly handles getting a ServerNotRunningException. See
+   * HBASE-4470. Note this doesn't check the exact exception thrown in the HBASE-4470 as there it is
+   * thrown from getHConnection() and here it is thrown from get() -- but those are both called from
+   * the same function anyway, and this way is less invasive than throwing from getHConnection would
+   * be.
+   */
+  @Test
+  public void testVerifyMetaRegionServerNotRunning()
+      throws IOException, InterruptedException, KeeperException, ServiceException {
+    testVerifyMetaRegionLocationWithException(new ServerNotRunningYetException("mock"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
index 53a757a..fb898ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
@@ -1,5 +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
@@ -18,16 +17,14 @@
  */
 package org.apache.hadoop.hbase;
 
+import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import java.io.IOException;
-
 /**
  * Defines a curated set of shared functions implemented by HBase servers (Masters
  * and RegionServers). For use internally only. Be judicious adding API. Changes cause ripples
@@ -64,14 +61,6 @@ public interface Server extends Abortable, Stoppable {
   ClusterConnection getClusterConnection();
 
   /**
-   * Returns instance of {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}
-   * running inside this server. This MetaServerLocator is started and stopped by server, clients
-   * shouldn't manage it's lifecycle.
-   * @return instance of {@link MetaTableLocator} associated with this server.
-   */
-  MetaTableLocator getMetaTableLocator();
-
-  /**
    * @return The unique server name for this server.
    */
   ServerName getServerName();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
index ce21465..e57817e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
@@ -65,8 +65,7 @@ class MasterMetaBootstrap {
       throw new IllegalStateException("hbase:meta must be initialized first before we can " +
           "assign out its replicas");
     }
-    ServerName metaServername =
-        this.master.getMetaTableLocator().getMetaRegionLocation(this.master.getZooKeeper());
+    ServerName metaServername = MetaTableLocator.getMetaRegionLocation(this.master.getZooKeeper());
     for (int i = 1; i < numReplicas; i++) {
       // Get current meta state for replica from zk.
       RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper(), i);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index fb76bec..cd838d5 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
@@ -102,6 +102,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -1550,9 +1551,9 @@ public class MasterRpcServices extends RSRpcServices
       }
       Pair<RegionInfo, ServerName> pair =
         MetaTableAccessor.getRegion(master.getConnection(), regionName);
-      if (Bytes.equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(),regionName)) {
+      if (Bytes.equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(), regionName)) {
         pair = new Pair<>(RegionInfoBuilder.FIRST_META_REGIONINFO,
-            master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper()));
+          MetaTableLocator.getMetaRegionLocation(master.getZooKeeper()));
       }
       if (pair == null) {
         throw new UnknownRegionException(Bytes.toString(regionName));

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java
index 560a04a..58e57c4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java
@@ -22,17 +22,15 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
-
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.tmpl.master.MasterStatusTmpl;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * The servlet responsible for rendering the index page of the
@@ -82,9 +80,7 @@ public class MasterStatusServlet extends HttpServlet {
   }
 
   private ServerName getMetaLocationOrNull(HMaster master) {
-    MetaTableLocator metaTableLocator = master.getMetaTableLocator();
-    return metaTableLocator == null ? null :
-      metaTableLocator.getMetaRegionLocation(master.getZooKeeper());
+    return MetaTableLocator.getMetaRegionLocation(master.getZooKeeper());
   }
 
   private Map<String, Integer> getFragmentationInfo(

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
index 9353124..46621da 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
@@ -25,7 +25,6 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -35,6 +34,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
@@ -225,12 +225,12 @@ public final class ProcedureSyncWait {
   protected static void waitMetaRegions(final MasterProcedureEnv env) throws IOException {
     int timeout = env.getMasterConfiguration().getInt("hbase.client.catalog.timeout", 10000);
     try {
-      if (env.getMasterServices().getMetaTableLocator().waitMetaRegionLocation(
-            env.getMasterServices().getZooKeeper(), timeout) == null) {
+      if (MetaTableLocator.waitMetaRegionLocation(env.getMasterServices().getZooKeeper(),
+        timeout) == null) {
         throw new NotAllMetaRegionsOnlineException();
       }
     } catch (InterruptedException e) {
-      throw (InterruptedIOException)new InterruptedIOException().initCause(e);
+      throw (InterruptedIOException) new InterruptedIOException().initCause(e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
index e78f0b5..c9dc0c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -42,6 +41,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@@ -159,7 +159,7 @@ public final class MasterSnapshotVerifier {
   private void verifyRegions(final SnapshotManifest manifest) throws IOException {
     List<RegionInfo> regions;
     if (TableName.META_TABLE_NAME.equals(tableName)) {
-      regions = new MetaTableLocator().getMetaRegions(services.getZooKeeper());
+      regions = MetaTableLocator.getMetaRegions(services.getZooKeeper());
     } else {
       regions = MetaTableAccessor.getTableRegions(services.getConnection(), tableName);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
index 1dce79f..3b7d65a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
@@ -24,7 +24,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CancellationException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -60,6 +59,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 
 /**
@@ -193,7 +193,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
 
       List<Pair<RegionInfo, ServerName>> regionsAndLocations;
       if (TableName.META_TABLE_NAME.equals(snapshotTable)) {
-        regionsAndLocations = new MetaTableLocator().getMetaRegionsAndLocations(
+        regionsAndLocations = MetaTableLocator.getMetaRegionsAndLocations(
           server.getZooKeeper());
       } else {
         regionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations(

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
index fee3dde..1f7a5e2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
@@ -24,7 +24,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ThreadPoolExecutor;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -43,14 +42,15 @@ import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
 import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
-import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 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.collect.Lists;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
 
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
@@ -131,7 +131,7 @@ public class MasterFlushTableProcedureManager extends MasterProcedureManager {
     List<Pair<RegionInfo, ServerName>> regionsAndLocations;
 
     if (TableName.META_TABLE_NAME.equals(tableName)) {
-      regionsAndLocations = new MetaTableLocator().getMetaRegionsAndLocations(
+      regionsAndLocations = MetaTableLocator.getMetaRegionsAndLocations(
         master.getZooKeeper());
     } else {
       regionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations(

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index b9d606d..6242d36 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -261,14 +261,6 @@ public class HRegionServer extends HasThread implements
    */
   protected ClusterConnection clusterConnection;
 
-  /*
-   * Long-living meta table locator, which is created when the server is started and stopped
-   * when server shuts down. References to this locator shall be used to perform according
-   * operations in EventHandlers. Primary reason for this decision is to make it mockable
-   * for tests.
-   */
-  protected MetaTableLocator metaTableLocator;
-
   /**
    * Go here to get table descriptors.
    */
@@ -836,7 +828,6 @@ public class HRegionServer extends HasThread implements
   protected synchronized void setupClusterConnection() throws IOException {
     if (clusterConnection == null) {
       clusterConnection = createClusterConnection();
-      metaTableLocator = new MetaTableLocator();
     }
   }
 
@@ -1104,8 +1095,6 @@ public class HRegionServer extends HasThread implements
       LOG.info("stopping server " + this.serverName);
     }
 
-    // so callers waiting for meta without timeout can stop
-    if (this.metaTableLocator != null) this.metaTableLocator.stop();
     if (this.clusterConnection != null && !clusterConnection.isClosed()) {
       try {
         this.clusterConnection.close();
@@ -2174,11 +2163,6 @@ public class HRegionServer extends HasThread implements
   }
 
   @Override
-  public MetaTableLocator getMetaTableLocator() {
-    return this.metaTableLocator;
-  }
-
-  @Override
   public void stop(final String msg) {
     stop(msg, false, RpcServer.getRequestUser().orElse(null));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index 62068fd..c7bccb3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -130,11 +129,6 @@ public class ReplicationSyncUp extends Configured implements Tool {
     }
 
     @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
     public ServerName getServerName() {
       return ServerName.valueOf(hostname, 1234, 1L);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index b43262d..14706c5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -134,7 +134,6 @@ import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALSplitter;
-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;
@@ -144,20 +143,20 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
 import org.apache.hbase.thirdparty.com.google.common.collect.Ordering;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hbase.thirdparty.com.google.common.collect.TreeMultimap;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -2002,11 +2001,6 @@ public class HBaseFsck extends Configured implements Closeable {
     });
   }
 
-  private ServerName getMetaRegionServerName(int replicaId)
-  throws IOException, KeeperException {
-    return new MetaTableLocator().getMetaRegionLocation(zkw, replicaId);
-  }
-
   /**
    * Contacts each regionserver and fetches metadata about regions.
    * @param regionServerList - the list of region servers to connect to

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index 372e5ca..5b996eb396 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -77,8 +77,6 @@
 <%
   HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
   Configuration conf = master.getConfiguration();
-
-  MetaTableLocator metaTableLocator = new MetaTableLocator();
   String fqtn = request.getParameter("name");
   final String escaped_fqtn = StringEscapeUtils.escapeHtml4(fqtn);
   Table table;
@@ -202,7 +200,7 @@ if ( fqtn != null ) {
   for (int j = 0; j < numMetaReplicas; j++) {
     RegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
                             RegionInfoBuilder.FIRST_META_REGIONINFO, j);
-    ServerName metaLocation = metaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1);
+    ServerName metaLocation = MetaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1);
     for (int i = 0; i < 1; i++) {
       String hostAndPort = "";
       String readReq = "N/A";

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index c0a2a8c..ff0a88c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -55,7 +55,6 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequester;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -162,11 +161,6 @@ public class MockRegionServerServices implements RegionServerServices {
   }
 
   @Override
-  public MetaTableLocator getMetaTableLocator() {
-    return null;
-  }
-
-  @Override
   public ZKWatcher getZooKeeper() {
     return zkw;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
index 0d21fb8..5582178 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
@@ -247,13 +247,11 @@ public class TestMetaTableAccessor {
     abstract void metaTask() throws Throwable;
   }
 
-  @Test public void testGetRegionsFromMetaTable()
-  throws IOException, InterruptedException {
-    List<RegionInfo> regions =
-      new MetaTableLocator().getMetaRegions(UTIL.getZooKeeperWatcher());
+  @Test
+  public void testGetRegionsFromMetaTable() throws IOException, InterruptedException {
+    List<RegionInfo> regions = MetaTableLocator.getMetaRegions(UTIL.getZooKeeperWatcher());
     assertTrue(regions.size() >= 1);
-    assertTrue(new MetaTableLocator().getMetaRegionsAndLocations(
-      UTIL.getZooKeeperWatcher()).size() >= 1);
+    assertTrue(MetaTableLocator.getMetaRegionsAndLocations(UTIL.getZooKeeperWatcher()).size() >= 1);
   }
 
   @Test public void testTableExists() throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
index db93c0c..9274fa0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
@@ -18,18 +18,10 @@
 package org.apache.hadoop.hbase;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.net.ConnectException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -51,8 +43,6 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse;
@@ -60,31 +50,34 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRespons
 /**
  * Test {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}
  */
-@Category({MiscTests.class, MediumTests.class})
+@Category({ MiscTests.class, MediumTests.class })
 public class TestMetaTableLocator {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestMetaTableLocator.class);
+    HBaseClassTestRule.forClass(TestMetaTableLocator.class);
 
   private static final Logger LOG = LoggerFactory.getLogger(TestMetaTableLocator.class);
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
   private static final ServerName SN =
-      ServerName.valueOf("example.org", 1234, System.currentTimeMillis());
+    ServerName.valueOf("example.org", 1234, System.currentTimeMillis());
   private ZKWatcher watcher;
   private Abortable abortable;
 
-  @BeforeClass public static void beforeClass() throws Exception {
+  @BeforeClass
+  public static void beforeClass() throws Exception {
     // Set this down so tests run quicker
     UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
     UTIL.startMiniZKCluster();
   }
 
-  @AfterClass public static void afterClass() throws IOException {
+  @AfterClass
+  public static void afterClass() throws IOException {
     UTIL.getZkCluster().shutdown();
   }
 
-  @Before public void before() throws IOException {
+  @Before
+  public void before() throws IOException {
     this.abortable = new Abortable() {
       @Override
       public void abort(String why, Throwable e) {
@@ -92,19 +85,20 @@ public class TestMetaTableLocator {
       }
 
       @Override
-      public boolean isAborted()  {
+      public boolean isAborted() {
         return false;
       }
     };
-    this.watcher = new ZKWatcher(UTIL.getConfiguration(),
-      this.getClass().getSimpleName(), this.abortable, true);
+    this.watcher =
+      new ZKWatcher(UTIL.getConfiguration(), this.getClass().getSimpleName(), this.abortable, true);
   }
 
-  @After public void after() {
+  @After
+  public void after() {
     try {
       // Clean out meta location or later tests will be confused... they presume
       // start fresh in zk.
-      new MetaTableLocator().deleteMetaLocation(this.watcher);
+      MetaTableLocator.deleteMetaLocation(this.watcher);
     } catch (KeeperException e) {
       LOG.warn("Unable to delete hbase:meta location", e);
     }
@@ -115,178 +109,47 @@ public class TestMetaTableLocator {
   /**
    * Test normal operations
    */
-  @Test public void testMetaLookup()
-          throws IOException, InterruptedException, ServiceException, KeeperException {
+  @Test
+  public void testMetaLookup()
+      throws IOException, InterruptedException, ServiceException, KeeperException {
     final ClientProtos.ClientService.BlockingInterface client =
-            Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
+      Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
 
-    Mockito.when(client.get((RpcController)Mockito.any(), (GetRequest)Mockito.any())).
-            thenReturn(GetResponse.newBuilder().build());
+    Mockito.when(client.get((RpcController) Mockito.any(), (GetRequest) Mockito.any()))
+      .thenReturn(GetResponse.newBuilder().build());
 
-    final MetaTableLocator mtl = new MetaTableLocator();
-    assertNull(mtl.getMetaRegionLocation(this.watcher));
+    assertNull(MetaTableLocator.getMetaRegionLocation(this.watcher));
     for (RegionState.State state : RegionState.State.values()) {
-      if (state.equals(RegionState.State.OPEN))
+      if (state.equals(RegionState.State.OPEN)) {
         continue;
+      }
       MetaTableLocator.setMetaLocation(this.watcher, SN, state);
-      assertNull(mtl.getMetaRegionLocation(this.watcher));
+      assertNull(MetaTableLocator.getMetaRegionLocation(this.watcher));
       assertEquals(state, MetaTableLocator.getMetaRegionState(this.watcher).getState());
     }
     MetaTableLocator.setMetaLocation(this.watcher, SN, RegionState.State.OPEN);
-    assertEquals(SN, mtl.getMetaRegionLocation(this.watcher));
+    assertEquals(SN, MetaTableLocator.getMetaRegionLocation(this.watcher));
     assertEquals(RegionState.State.OPEN,
       MetaTableLocator.getMetaRegionState(this.watcher).getState());
 
-    mtl.deleteMetaLocation(this.watcher);
+    MetaTableLocator.deleteMetaLocation(this.watcher);
     assertNull(MetaTableLocator.getMetaRegionState(this.watcher).getServerName());
     assertEquals(RegionState.State.OFFLINE,
-        MetaTableLocator.getMetaRegionState(this.watcher).getState());
-    assertNull(mtl.getMetaRegionLocation(this.watcher));
-  }
-
-
-  /**
-   * Test interruptable while blocking wait on meta.
-   * @throws IOException
-   * @throws ServiceException
-   * @throws InterruptedException
-   */
-  @Test public void testInterruptWaitOnMeta()
-  throws IOException, InterruptedException, ServiceException {
-    final ClientProtos.ClientService.BlockingInterface client =
-      Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
-
-    Mockito.when(client.get((RpcController)Mockito.any(), (GetRequest)Mockito.any())).
-    thenReturn(GetResponse.newBuilder().build());
-
-    final MetaTableLocator mtl = new MetaTableLocator();
-    ServerName meta = new MetaTableLocator().getMetaRegionLocation(this.watcher);
-    assertNull(meta);
-    Thread t = new Thread() {
-      @Override
-      public void run() {
-        try {
-          mtl.waitMetaRegionLocation(watcher);
-        } catch (InterruptedException e) {
-          throw new RuntimeException("Interrupted", e);
-        }
-      }
-    };
-    t.start();
-    while (!t.isAlive())
-      Threads.sleep(1);
-    Threads.sleep(1);
-    assertTrue(t.isAlive());
-    mtl.stop();
-    // Join the thread... should exit shortly.
-    t.join();
-  }
-
-  private void testVerifyMetaRegionLocationWithException(Exception ex)
-  throws IOException, InterruptedException, KeeperException, ServiceException {
-    // Mock an ClientProtocol.
-    final ClientProtos.ClientService.BlockingInterface implementation =
-      Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
-
-    ClusterConnection connection = mockConnection(null, implementation);
-
-    // If a 'get' is called on mocked interface, throw connection refused.
-    Mockito.when(implementation.get((RpcController) Mockito.any(), (GetRequest) Mockito.any())).
-      thenThrow(new ServiceException(ex));
-
-    long timeout = UTIL.getConfiguration().
-            getLong("hbase.catalog.verification.timeout", 1000);
-    MetaTableLocator.setMetaLocation(this.watcher, SN, RegionState.State.OPENING);
-    assertFalse(new MetaTableLocator().verifyMetaRegionLocation(
-      connection, watcher, timeout));
-
-    MetaTableLocator.setMetaLocation(this.watcher, SN, RegionState.State.OPEN);
-    assertFalse(new MetaTableLocator().verifyMetaRegionLocation(
-            connection, watcher, timeout));
-  }
-
-  /**
-   * Test we survive a connection refused {@link ConnectException}
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws KeeperException
-   * @throws ServiceException
-   */
-  @Test
-  public void testGetMetaServerConnectionFails()
-  throws IOException, InterruptedException, KeeperException, ServiceException {
-    testVerifyMetaRegionLocationWithException(new ConnectException("Connection refused"));
-  }
-
-  /**
-   * Test that verifyMetaRegionLocation properly handles getting a
-   * ServerNotRunningException. See HBASE-4470.
-   * Note this doesn't check the exact exception thrown in the
-   * HBASE-4470 as there it is thrown from getHConnection() and
-   * here it is thrown from get() -- but those are both called
-   * from the same function anyway, and this way is less invasive than
-   * throwing from getHConnection would be.
-   *
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws KeeperException
-   * @throws ServiceException
-   */
-  @Test
-  public void testVerifyMetaRegionServerNotRunning()
-  throws IOException, InterruptedException, KeeperException, ServiceException {
-    testVerifyMetaRegionLocationWithException(new ServerNotRunningYetException("mock"));
-  }
-
-  /**
-   * Test get of meta region fails properly if nothing to connect to.
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws KeeperException
-   * @throws ServiceException
-   */
-  @Test
-  public void testVerifyMetaRegionLocationFails()
-  throws IOException, InterruptedException, KeeperException, ServiceException {
-    ClusterConnection connection = Mockito.mock(ClusterConnection.class);
-    ServiceException connectException =
-      new ServiceException(new ConnectException("Connection refused"));
-    final AdminProtos.AdminService.BlockingInterface implementation =
-      Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
-    Mockito.when(implementation.getRegionInfo((RpcController)Mockito.any(),
-      (GetRegionInfoRequest)Mockito.any())).thenThrow(connectException);
-    Mockito.when(connection.getAdmin(Mockito.any())).
-      thenReturn(implementation);
-        RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class);
-        Mockito.when(controllerFactory.newController()).thenReturn(
-          Mockito.mock(HBaseRpcController.class));
-        Mockito.when(connection.getRpcControllerFactory()).thenReturn(controllerFactory);
-
-    ServerName sn = ServerName.valueOf("example.com", 1234, System.currentTimeMillis());
-    MetaTableLocator.setMetaLocation(this.watcher,
-            sn,
-            RegionState.State.OPENING);
-    assertFalse(new MetaTableLocator().verifyMetaRegionLocation(connection, watcher, 100));
-    MetaTableLocator.setMetaLocation(this.watcher, sn, RegionState.State.OPEN);
-    assertFalse(new MetaTableLocator().verifyMetaRegionLocation(connection, watcher, 100));
+      MetaTableLocator.getMetaRegionState(this.watcher).getState());
+    assertNull(MetaTableLocator.getMetaRegionLocation(this.watcher));
   }
 
-  @Test (expected = NotAllMetaRegionsOnlineException.class)
-  public void testTimeoutWaitForMeta()
-  throws IOException, InterruptedException {
-    new MetaTableLocator().waitMetaRegionLocation(watcher, 100);
+  @Test(expected = NotAllMetaRegionsOnlineException.class)
+  public void testTimeoutWaitForMeta() throws IOException, InterruptedException {
+    MetaTableLocator.waitMetaRegionLocation(watcher, 100);
   }
 
   /**
    * Test waiting on meat w/ no timeout specified.
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws KeeperException
    */
-  @Test public void testNoTimeoutWaitForMeta()
-  throws IOException, InterruptedException, KeeperException {
-    final MetaTableLocator mtl = new MetaTableLocator();
-    ServerName hsa = mtl.getMetaRegionLocation(watcher);
+  @Test
+  public void testNoTimeoutWaitForMeta() throws IOException, InterruptedException, KeeperException {
+    ServerName hsa = MetaTableLocator.getMetaRegionLocation(watcher);
     assertNull(hsa);
 
     // Now test waiting on meta location getting set.
@@ -298,54 +161,12 @@ public class TestMetaTableLocator {
     // Join the thread... should exit shortly.
     t.join();
     // Now meta is available.
-    assertTrue(mtl.getMetaRegionLocation(watcher).equals(hsa));
-  }
-
-  /**
-   * @param admin An {@link AdminProtos.AdminService.BlockingInterface} instance; you'll likely
-   * want to pass a mocked HRS; can be null.
-   * @param client A mocked ClientProtocol instance, can be null
-   * @return Mock up a connection that returns a {@link Configuration} when
-   * {@link org.apache.hadoop.hbase.client.ClusterConnection#getConfiguration()} is called, a 'location' when
-   * {@link org.apache.hadoop.hbase.client.RegionLocator#getRegionLocation(byte[], boolean)} is called,
-   * and that returns the passed {@link AdminProtos.AdminService.BlockingInterface} instance when
-   * {@link org.apache.hadoop.hbase.client.ClusterConnection#getAdmin(ServerName)} is called, returns the passed
-   * {@link ClientProtos.ClientService.BlockingInterface} instance when
-   * {@link org.apache.hadoop.hbase.client.ClusterConnection#getClient(ServerName)} is called.
-   * @throws IOException
-   */
-  private ClusterConnection mockConnection(final AdminProtos.AdminService.BlockingInterface admin,
-      final ClientProtos.ClientService.BlockingInterface client)
-  throws IOException {
-    ClusterConnection connection =
-      HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
-    Mockito.doNothing().when(connection).close();
-    // Make it so we return any old location when asked.
-    final HRegionLocation anyLocation = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, SN);
-    Mockito.when(connection.getRegionLocation((TableName) Mockito.any(),
-        (byte[]) Mockito.any(), Mockito.anyBoolean())).
-      thenReturn(anyLocation);
-    Mockito.when(connection.locateRegion((TableName) Mockito.any(),
-        (byte[]) Mockito.any())).
-      thenReturn(anyLocation);
-    if (admin != null) {
-      // If a call to getHRegionConnection, return this implementation.
-      Mockito.when(connection.getAdmin(Mockito.any())).
-        thenReturn(admin);
-    }
-    if (client != null) {
-      // If a call to getClient, return this implementation.
-      Mockito.when(connection.getClient(Mockito.any())).
-        thenReturn(client);
-    }
-    return connection;
+    assertTrue(MetaTableLocator.getMetaRegionLocation(watcher).equals(hsa));
   }
 
   private void startWaitAliveThenWaitItLives(final Thread t, final int ms) {
     t.start();
-    while(!t.isAlive()) {
-      // Wait
-    }
+    UTIL.waitFor(2000, t::isAlive);
     // Wait one second.
     Threads.sleep(ms);
     assertTrue("Assert " + t.getName() + " still waiting", t.isAlive());
@@ -372,9 +193,13 @@ public class TestMetaTableLocator {
 
     void doWaiting() throws InterruptedException {
       try {
-        while (new MetaTableLocator().waitMetaRegionLocation(watcher, 10000) == null);
+        for (;;) {
+          if (MetaTableLocator.waitMetaRegionLocation(watcher, 10000) != null) {
+            break;
+          }
+        }
       } catch (NotAllMetaRegionsOnlineException e) {
-        //Ignore
+        // Ignore
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index bac588a..3b14b7f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 import org.apache.hadoop.hbase.util.HBaseFsckRepair;
 import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
 import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
+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;
@@ -100,8 +101,7 @@ public class TestMetaWithReplicas {
     AssignmentManager am = TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager();
     Set<ServerName> sns = new HashSet<ServerName>();
     ServerName hbaseMetaServerName =
-        TEST_UTIL.getMiniHBaseCluster().getMaster().getMetaTableLocator().
-            getMetaRegionLocation(TEST_UTIL.getZooKeeperWatcher());
+      MetaTableLocator.getMetaRegionLocation(TEST_UTIL.getZooKeeperWatcher());
     LOG.info("HBASE:META DEPLOY: on " + hbaseMetaServerName);
     sns.add(hbaseMetaServerName);
     for (int replicaId = 1; replicaId < 3; replicaId++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index ac20dbd..9c55f57 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 
 public class MockNoopMasterServices implements MasterServices {
@@ -161,11 +160,6 @@ public class MockNoopMasterServices implements MasterServices {
   }
 
   @Override
-  public MetaTableLocator getMetaTableLocator() {
-    return null;
-  }
-
-  @Override
   public ClusterConnection getConnection() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index f4c2a33..844b705 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -68,7 +68,6 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequester;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
@@ -305,11 +304,6 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
   }
 
   @Override
-  public MetaTableLocator getMetaTableLocator() {
-    return null;
-  }
-
-  @Override
   public ServerName getServerName() {
     return this.sn;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
----------------------------------------------------------------------
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 5851dea..2300f54 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
@@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKListener;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -317,11 +316,6 @@ public class TestActiveMasterManager {
       return null;
     }
 
-    @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
     public ClusterStatusTracker getClusterStatusTracker() {
       return clusterStatusTracker;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 9da4df4..5c8db3e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -233,11 +232,6 @@ public class TestHFileCleaner {
     }
 
     @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
     public ServerName getServerName() {
       return ServerName.valueOf("regionserver,60020,000000");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
index c011ea8..119194b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -169,11 +168,6 @@ public class TestHFileLinkCleaner {
     }
 
     @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
     public ServerName getServerName() {
       return ServerName.valueOf("regionserver,60020,000000");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 882ea9d..247ed01 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -30,7 +30,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -55,7 +54,6 @@ import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
@@ -367,11 +365,6 @@ public class TestLogsCleaner {
     }
 
     @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
     public ServerName getServerName() {
       return ServerName.valueOf("regionserver,60020,000000");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index 24b930c..d162bf3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -55,7 +55,6 @@ import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
@@ -257,11 +256,6 @@ public class TestReplicationHFileCleaner {
     }
 
     @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
     public ServerName getServerName() {
       return ServerName.valueOf("regionserver,60020,000000");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
index 9f05a73..8c9ce75 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
@@ -49,7 +49,6 @@ import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerContext;
 import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerResult;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -834,11 +833,6 @@ public class TestHeapMemoryManager {
     }
 
     @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
     public ServerName getServerName() {
       return ServerName.valueOf("server1",4000,12345);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
index af2861f..b9f89b7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
@@ -111,8 +111,7 @@ public class TestRegionServerNoMaster {
     HRegionServer hrs = HTU.getHBaseCluster()
       .getLiveRegionServerThreads().get(0).getRegionServer();
     ZKWatcher zkw = hrs.getZooKeeper();
-    MetaTableLocator mtl = new MetaTableLocator();
-    ServerName sn = mtl.getMetaRegionLocation(zkw);
+    ServerName sn = MetaTableLocator.getMetaRegionLocation(zkw);
     if (sn != null && !masterAddr.equals(sn)) {
       return;
     }
@@ -120,7 +119,7 @@ public class TestRegionServerNoMaster {
     ProtobufUtil.openRegion(null, hrs.getRSRpcServices(),
       hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
     while (true) {
-      sn = mtl.getMetaRegionLocation(zkw);
+      sn = MetaTableLocator.getMetaRegionLocation(zkw);
       if (sn != null && sn.equals(hrs.getServerName())
           && hrs.onlineRegions.containsKey(
               HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
index 0c38ee3..cbf932c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
@@ -49,7 +49,6 @@ import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -136,11 +135,6 @@ public class TestSplitLogWorker {
     }
 
     @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
     public ChoreService getChoreService() {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 84b8d6c..0e20252 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.After;
 import org.junit.Assert;
@@ -475,11 +474,6 @@ public class TestWALLockup {
     }
 
     @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
     public ServerName getServerName() {
       return ServerName.valueOf(this.serverName);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index 2419095..863d558 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -214,11 +213,6 @@ public class TestReplicationTrackerZKImpl {
     }
 
     @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
     public ServerName getServerName() {
       return ServerName.valueOf(this.serverName);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 0872ea7..86bbb09 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -88,7 +88,6 @@ import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -858,11 +857,6 @@ public abstract class TestReplicationSourceManager {
     }
 
     @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
     public ServerName getServerName() {
       return ServerName.valueOf(hostname, 1234, 1L);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
index afde037..e4780f1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
@@ -70,7 +70,6 @@ import org.apache.hadoop.hbase.util.Sleeper;
 import org.apache.hadoop.hbase.util.Strings;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.net.DNS;
@@ -216,11 +215,6 @@ public class TestTokenAuthentication {
     }
 
     @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
     public ZKWatcher getZooKeeper() {
       return zookeeper;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bf966c8/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
index 551b940..c25db01 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.util;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.ChoreService;
@@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -110,11 +108,6 @@ public class MockServer implements Server {
   }
 
   @Override
-  public MetaTableLocator getMetaTableLocator() {
-    return null;
-  }
-
-  @Override
   public ServerName getServerName() {
     return NAME;
   }