You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nd...@apache.org on 2020/01/14 16:30:36 UTC

[hbase] branch HBASE-18095/client-locate-meta-no-zookeeper updated (4a74bb3 -> d9bb034)

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

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


 discard 4a74bb3  HBASE-23305: Master based registry implementation (#954)
    omit d2d7b2b  HBASE-23648: Re-use underlying connection registry in RawAsyncHBaseAdmin (#994)
    omit d9c17c3  HBASE-23604: Clarify AsyncRegistry usage in the code. (#957)
    omit 02ad1eb  HBASE-23304: RPCs needed for client meta information lookup (#904)
    omit 7b0e854  HBASE-23281: Track meta region locations in masters (#830)
    omit 113e521  HBASE-23275: Track active master's address in ActiveMasterManager (#812)
     add 12b79a3  HBASE-22285 A normalizer which merges small size regions with adjacent regions (#978)
     add f3bdb02  HBASE-23378: Clean Up FSUtil setClusterId (#910)
     add 9abdb7b  HBASE-23055 Alter hbase:meta
     add 56842d0  HBASE-23659 BaseLoadBalancer#wouldLowerAvailability should consider region replicas (#1001)
     add 8ca6148  HBASE-23668 Master log start filling with "Flush journal status" messages"
     add 4ed4669  Revert "HBASE-23668 Master log start filling with "Flush journal status" messages" Minor addendum fixing log message.
     add 2e0edac  HBASE-23662 : Replace HColumnDescriptor(String cf) with ColumnFamilyDescriptor
     add c473f35  HBASE-23675 Move to Apache parent POM version 22
     add 79e799a  HBASE-23165 [hbtop] Some modifications from HBASE-22988 (#987)
     add c8496bc  Revert "HBASE-23055 Alter hbase:meta"
     add 0a9e1f8  HBASE-23646 Resolved remaining Checkstyle violations in tests of hbase-rest
     add 4ad12e0  HBASE-23681 Add UT for procedure store region flusher (#1024)
     add 0bf933b  HBASE-23662 : Keep HCD constructor until shell usages are replaced
     add 1d2c3ef  HBASE-23679 FileSystem objects leak when cleaned up in cleanupBulkLoad
     add 499ff32  HBASE-23383 [hbck2] `fixHoles` should queue assignment procedures for any regions its fixing (#917)
     new 9c42b6a  HBASE-23275: Track active master's address in ActiveMasterManager (#812)
     new 8d4314f  HBASE-23281: Track meta region locations in masters (#830)
     new bc891a8  HBASE-23304: RPCs needed for client meta information lookup (#904)
     new e2a9f11  HBASE-23604: Clarify AsyncRegistry usage in the code. (#957)
     new 49fb451  HBASE-23648: Re-use underlying connection registry in RawAsyncHBaseAdmin (#994)
     new d9bb034  HBASE-23305: Master based registry implementation (#954)

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (4a74bb3)
            \
             N -- N -- N   refs/heads/HBASE-18095/client-locate-meta-no-zookeeper (d9bb034)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 bin/hbase                                          |   5 +-
 conf/log4j-hbtop.properties                        |   2 +-
 hbase-build-configuration/pom.xml                  |   1 -
 .../resources/hbase/checkstyle-suppressions.xml    |   2 +
 .../org/apache/hadoop/hbase/HColumnDescriptor.java |   2 +-
 .../apache/hadoop/hbase/TestHColumnDescriptor.java |  49 ---
 hbase-common/src/main/resources/hbase-default.xml  |   6 +
 .../hadoop/hbase/coprocessor/TestClassLoading.java | 110 ++++--
 .../apache/hadoop/hbase/hbtop/RecordFilter.java    |   3 +
 .../hbase/hbtop/mode/RequestCountPerSecond.java    |  13 +-
 .../hbase/hbtop/screen/field/FieldScreenView.java  |   4 +-
 .../hbase/hbtop/screen/top/TopScreenPresenter.java |   4 +-
 .../hbtop/{RecordTest.java => TestRecord.java}     |   4 +-
 ...RecordFilterTest.java => TestRecordFilter.java} |   4 +-
 .../org/apache/hadoop/hbase/hbtop/TestUtils.java   |   4 +-
 .../{FieldValueTest.java => TestFieldValue.java}   |   4 +-
 .../{ClientModeTest.java => TestClientMode.java}   |   4 +-
 .../mode/{ModeTestBase.java => TestModeBase.java}  |   2 +-
 ...mespaceModeTest.java => TestNamespaceMode.java} |   4 +-
 .../{RegionModeTest.java => TestRegionMode.java}   |   4 +-
 ...rverModeTest.java => TestRegionServerMode.java} |   4 +-
 ...ondTest.java => TestRequestCountPerSecond.java} |   4 +-
 .../{TableModeTest.java => TestTableMode.java}     |   4 +-
 .../mode/{UserModeTest.java => TestUserMode.java}  |   4 +-
 ...nterTest.java => TestFieldScreenPresenter.java} |   4 +-
 ...enterTest.java => TestHelpScreenPresenter.java} |   4 +-
 ...enterTest.java => TestModeScreenPresenter.java} |   4 +-
 ...a => TestFilterDisplayModeScreenPresenter.java} |   4 +-
 ...Test.java => TestInputModeScreenPresenter.java} |   4 +-
 ...st.java => TestMessageModeScreenPresenter.java} |   4 +-
 .../top/{PagingTest.java => TestPaging.java}       |   4 +-
 ...creenModelTest.java => TestTopScreenModel.java} |   4 +-
 ...senterTest.java => TestTopScreenPresenter.java} |   4 +-
 .../{CursorTest.java => impl/TestCursor.java}      |  10 +-
 .../{KeyPressTest.java => impl/TestKeyPress.java}  |  10 +-
 .../TestTerminalPrinter.java}                      |  11 +-
 .../hadoop/hbase/IntegrationTestManyRegions.java   |  17 +-
 .../hbase/mapreduce/TestHFileOutputFormat2.java    |  88 +++--
 .../apache/hadoop/hbase/rest/SchemaResource.java   |  67 ++--
 .../hadoop/hbase/rest/model/TableSchemaModel.java  |  19 +-
 .../hadoop/hbase/rest/HBaseRESTTestingUtility.java |  15 +-
 .../hadoop/hbase/rest/PerformanceEvaluation.java   |  68 ++--
 .../apache/hadoop/hbase/rest/RowResourceBase.java  | 119 +++---
 .../apache/hadoop/hbase/rest/TestGzipFilter.java   |  14 +-
 .../hadoop/hbase/rest/TestMultiRowResource.java    |  49 ++-
 .../hbase/rest/TestNamespacesInstanceResource.java |  21 +-
 .../hadoop/hbase/rest/TestScannerResource.java     |  32 +-
 .../hadoop/hbase/rest/TestScannersWithFilters.java | 415 +++++++++++----------
 .../hadoop/hbase/rest/TestScannersWithLabels.java  |  18 +-
 .../apache/hadoop/hbase/rest/TestTableScan.java    |  18 +-
 .../hadoop/hbase/rest/client/TestRemoteTable.java  |   2 +-
 .../org/apache/hadoop/hbase/master/MetaFixer.java  | 214 ++++++++---
 .../hbase/master/balancer/BaseLoadBalancer.java    | 106 +++---
 .../normalizer/AbstractRegionNormalizer.java       | 213 +++++++++++
 .../hbase/master/normalizer/MergeNormalizer.java   | 143 +++++++
 .../master/normalizer/SimpleRegionNormalizer.java  | 180 ++-------
 .../store/region/RegionFlusherAndCompactor.java    |  17 +-
 .../store/region/RegionProcedureStore.java         |  11 +-
 .../apache/hadoop/hbase/regionserver/HRegion.java  |   4 +-
 .../hbase/regionserver/SecureBulkLoadManager.java  |  37 +-
 .../org/apache/hadoop/hbase/tool/CanaryTool.java   |  15 +-
 .../java/org/apache/hadoop/hbase/util/FSUtils.java |  76 ++--
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |  19 +-
 .../org/apache/hadoop/hbase/TestNamespace.java     |  75 ++--
 .../org/apache/hadoop/hbase/TestSerialization.java |  17 +-
 .../hbase/client/RegionReplicaTestHelper.java      |  35 +-
 .../org/apache/hadoop/hbase/client/TestAdmin2.java |  20 +-
 .../hbase/client/TestAsyncNamespaceAdminApi.java   |  14 +-
 .../hbase/coprocessor/TestCoprocessorMetrics.java  |  22 +-
 .../hbase/filter/TestColumnPrefixFilter.java       |  36 +-
 .../filter/TestMultipleColumnPrefixFilter.java     |  60 +--
 .../hbase/io/encoding/TestChangingEncoding.java    |  20 +-
 .../hadoop/hbase/io/hfile/TestCacheConfig.java     |  13 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java    |  14 +-
 .../org/apache/hadoop/hbase/master/TestMaster.java |  23 +-
 .../apache/hadoop/hbase/master/TestMetaFixer.java  |  49 ++-
 .../master/assignment/TestRegionReplicaSplit.java  |  39 +-
 .../master/normalizer/TestMergeNormalizer.java     | 270 ++++++++++++++
 .../master/procedure/TestModifyTableProcedure.java |  72 ++--
 .../hadoop/hbase/master/procedure/TestSCPBase.java |  39 +-
 .../hbase/mob/TestExpiredMobFileCleaner.java       |  35 +-
 .../apache/hadoop/hbase/mob/TestMobFileCache.java  |  75 ++--
 .../hbase/mob/compactions/TestMobCompactor.java    | 102 ++---
 .../compactions/TestPartitionedMobCompactor.java   |  27 +-
 .../hbase/namespace/TestNamespaceAuditor.java      | 153 ++++----
 .../region/TestRegionProcedureStoreFlush.java      | 150 ++++++++
 .../hbase/quotas/TestMasterQuotasObserver.java     |  15 +-
 .../hadoop/hbase/quotas/TestRegionSizeUse.java     |  16 +-
 .../hbase/regionserver/TestAtomicOperation.java    |  12 +-
 .../hbase/regionserver/TestColumnSeeking.java      |  16 +-
 .../regionserver/TestEncryptionKeyRotation.java    |  81 ++--
 .../regionserver/TestEncryptionRandomKeying.java   |  34 +-
 .../hadoop/hbase/regionserver/TestHRegion.java     |  31 +-
 .../hbase/regionserver/TestHRegionFileSystem.java  |  16 +-
 .../regionserver/TestHRegionServerBulkLoad.java    |  17 +-
 .../regionserver/TestMemStoreSegmentsIterator.java |  19 +-
 .../hbase/regionserver/TestScanWithBloomError.java |  15 +-
 .../regionserver/TestScannerWithBulkload.java      |  18 +-
 .../hbase/regionserver/TestSeekOptimizations.java  |  18 +-
 .../regionserver/TestTimestampFilterSeekHint.java  |  17 +-
 .../security/access/TestAccessController.java      |  16 +-
 .../security/visibility/TestVisibilityLabels.java  |  21 +-
 .../hadoop/hbase/util/BaseTestHBaseFsck.java       |  43 ++-
 .../hadoop/hbase/util/TestHBaseFsckEncryption.java |  40 +-
 .../util/hbck/OfflineMetaRebuildTestCore.java      |  16 +-
 .../hadoop/hbase/thrift/TestThriftServer.java      |  20 +-
 pom.xml                                            |   4 +-
 107 files changed, 2644 insertions(+), 1512 deletions(-)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/{RecordTest.java => TestRecord.java} (97%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/{RecordFilterTest.java => TestRecordFilter.java} (98%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/field/{FieldValueTest.java => TestFieldValue.java} (99%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/mode/{ClientModeTest.java => TestClientMode.java} (95%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/mode/{ModeTestBase.java => TestModeBase.java} (97%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/mode/{NamespaceModeTest.java => TestNamespaceMode.java} (95%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/mode/{RegionModeTest.java => TestRegionMode.java} (94%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/mode/{RegionServerModeTest.java => TestRegionServerMode.java} (95%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/mode/{RequestCountPerSecondTest.java => TestRequestCountPerSecond.java} (96%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/mode/{TableModeTest.java => TestTableMode.java} (96%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/mode/{UserModeTest.java => TestUserMode.java} (95%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/screen/field/{FieldScreenPresenterTest.java => TestFieldScreenPresenter.java} (98%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/screen/help/{HelpScreenPresenterTest.java => TestHelpScreenPresenter.java} (95%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/screen/mode/{ModeScreenPresenterTest.java => TestModeScreenPresenter.java} (97%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/screen/top/{FilterDisplayModeScreenPresenterTest.java => TestFilterDisplayModeScreenPresenter.java} (95%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/screen/top/{InputModeScreenPresenterTest.java => TestInputModeScreenPresenter.java} (98%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/screen/top/{MessageModeScreenPresenterTest.java => TestMessageModeScreenPresenter.java} (95%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/screen/top/{PagingTest.java => TestPaging.java} (98%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/screen/top/{TopScreenModelTest.java => TestTopScreenModel.java} (98%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/screen/top/{TopScreenPresenterTest.java => TestTopScreenPresenter.java} (98%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/terminal/{CursorTest.java => impl/TestCursor.java} (90%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/terminal/{KeyPressTest.java => impl/TestKeyPress.java} (86%)
 rename hbase-hbtop/src/test/java/org/apache/hadoop/hbase/hbtop/terminal/{TerminalPrinterTest.java => impl/TestTerminalPrinter.java} (85%)
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/AbstractRegionNormalizer.java
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizer.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestMergeNormalizer.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStoreFlush.java


[hbase] 06/06: HBASE-23305: Master based registry implementation (#954)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d9bb034c9439d034f23dd0b66faafce9726e2fa5
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Tue Jan 14 08:24:07 2020 -0800

    HBASE-23305: Master based registry implementation (#954)
    
    Implements a master based registry for clients.
    
     - Supports hedged RPCs (fan out configured via configs).
     - Parameterized existing client tests to run with multiple registry combinations.
     - Added unit-test coverage for the new registry implementation.
    
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
    Signed-off-by: stack <st...@apache.org>
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../hbase/client/ConnectionRegistryFactory.java    |   4 +-
 .../apache/hadoop/hbase/client/MasterRegistry.java | 226 +++++++++++
 .../MasterRegistryFetchException.java}             |  32 +-
 .../apache/hadoop/hbase/ipc/AbstractRpcClient.java |  57 +--
 .../apache/hadoop/hbase/ipc/BlockingRpcClient.java |   7 +-
 .../apache/hadoop/hbase/ipc/HedgedRpcChannel.java  | 274 ++++++++++++++
 .../apache/hadoop/hbase/ipc/NettyRpcClient.java    |  34 +-
 .../org/apache/hadoop/hbase/ipc/RpcClient.java     |  19 +-
 .../hbase/client/TestConnectionRegistryLeak.java   |   3 +-
 .../java/org/apache/hadoop/hbase/HConstants.java   |  20 +-
 .../apache/hadoop/hbase/util/PrettyPrinter.java    |  23 +-
 .../org/apache/hadoop/hbase/TestTableName.java     |  16 +-
 .../apache/hadoop/hbase/util/JVMClusterUtil.java   |   7 +
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |   3 +
 .../hbase/client/DummyConnectionRegistry.java      |   3 +-
 .../hadoop/hbase/client/TestFromClientSide.java    | 420 +++++++++++----------
 .../client/TestFromClientSideWithCoprocessor.java  |  23 +-
 .../hadoop/hbase/client/TestMasterRegistry.java    | 125 ++++++
 .../hbase/client/TestScannersFromClientSide.java   | 136 ++++---
 .../apache/hadoop/hbase/ipc/AbstractTestIPC.java   | 120 +++++-
 .../hbase/ipc/TestProtobufRpcServiceImpl.java      |  25 +-
 21 files changed, 1243 insertions(+), 334 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
index 80d358b..9308443 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -27,9 +28,6 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 final class ConnectionRegistryFactory {
 
-  static final String CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY =
-      "hbase.client.connection.registry.impl";
-
   private ConnectionRegistryFactory() {
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
new file mode 100644
index 0000000..5680847
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.exceptions.MasterRegistryFetchException;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.hadoop.hbase.ipc.RpcClientFactory;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClientMetaService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse;
+
+/**
+ * Master based registry implementation. Makes RPCs to the configured master addresses from config
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements ConnectionRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final Set<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new HashSet<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
+      HostAndPort masterHostPort =
+          HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
+      masterServers.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
+    }
+    Preconditions.checkArgument(!masterServers.isEmpty(), "At least one master address is needed");
+  }
+
+  @VisibleForTesting
+  public Set<ServerName> getParsedMasterServers() {
+    return Collections.unmodifiableSet(masterServers);
+  }
+
+  /**
+   * Returns a call back that can be passed along to the non-blocking rpc call. It is invoked once
+   * the rpc finishes and the response is propagated to the passed future.
+   * @param future Result future to which the rpc response is propagated.
+   * @param isValidResp Checks if the rpc response has a valid result.
+   * @param transformResult Transforms the result to a different form as expected by callers.
+   * @param hrc RpcController instance for this rpc.
+   * @param debug Debug message passed along to the caller in case of exceptions.
+   * @param <T> RPC result type.
+   * @param <R> Transformed type of the result.
+   * @return A call back that can be embedded in the non-blocking rpc call.
+   */
+  private <T, R> RpcCallback<T> getRpcCallBack(CompletableFuture<R> future,
+      Predicate<T> isValidResp, Function<T, R> transformResult, HBaseRpcController hrc,
+      final String debug) {
+    return rpcResult -> {
+      if (rpcResult == null) {
+        future.completeExceptionally(
+            new MasterRegistryFetchException(masterServers, hrc.getFailed()));
+      }
+      if (!isValidResp.test(rpcResult)) {
+        // Rpc returned ok, but result was malformed.
+        future.completeExceptionally(new IOException(
+            String.format("Invalid result for request %s. Will be retried", debug)));
+
+      }
+      future.complete(transformResult.apply(rpcResult));
+    };
+  }
+
+  /**
+   * Simple helper to transform the result of getMetaRegionLocations() rpc.
+   */
+  private RegionLocations transformMetaRegionLocations(GetMetaRegionLocationsResponse resp) {
+    List<HRegionLocation> regionLocations = new ArrayList<>();
+    resp.getMetaLocationsList().forEach(
+      location -> regionLocations.add(ProtobufUtil.toRegionLocation(location)));
+    return new RegionLocations(regionLocations);
+  }
+
+  @Override
+  public CompletableFuture<RegionLocations> getMetaRegionLocations() {
+    CompletableFuture<RegionLocations> result = new CompletableFuture<>();
+    HBaseRpcController hrc = rpcControllerFactory.newController();
+    RpcCallback<GetMetaRegionLocationsResponse> callback = getRpcCallBack(result,
+      (rpcResp) -> rpcResp.getMetaLocationsCount() != 0, this::transformMetaRegionLocations, hrc,
+        "getMetaRegionLocations()");
+    try {
+      getMasterStub().getMetaRegionLocations(
+          hrc, GetMetaRegionLocationsRequest.getDefaultInstance(), callback);
+    } catch (IOException e) {
+      result.completeExceptionally(e);
+    }
+    return result;
+  }
+
+  @Override
+  public CompletableFuture<String> getClusterId() {
+    CompletableFuture<String> result = new CompletableFuture<>();
+    HBaseRpcController hrc = rpcControllerFactory.newController();
+    RpcCallback<GetClusterIdResponse> callback = getRpcCallBack(result,
+        GetClusterIdResponse::hasClusterId,  GetClusterIdResponse::getClusterId, hrc,
+        "getClusterId()");
+    try {
+      getMasterStub().getClusterId(hrc, GetClusterIdRequest.getDefaultInstance(), callback);
+    } catch (IOException e) {
+      result.completeExceptionally(e);
+    }
+    return result;
+  }
+
+  private ServerName transformServerName(GetActiveMasterResponse resp) {
+    return ProtobufUtil.toServerName(resp.getServerName());
+  }
+
+  @Override
+  public CompletableFuture<ServerName> getActiveMaster() {
+    CompletableFuture<ServerName> result = new CompletableFuture<>();
+    HBaseRpcController hrc = rpcControllerFactory.newController();
+    RpcCallback<GetActiveMasterResponse> callback = getRpcCallBack(result,
+        GetActiveMasterResponse::hasServerName, this::transformServerName, hrc,
+        "getActiveMaster()");
+    try {
+      getMasterStub().getActiveMaster(hrc, GetActiveMasterRequest.getDefaultInstance(), callback);
+    } catch (IOException e) {
+      result.completeExceptionally(e);
+    }
+    return result;
+  }
+
+  @Override
+  public void close() {
+    if (rpcClient != null) {
+      rpcClient.close();
+    }
+  }
+}
\ No newline at end of file
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
similarity index 53%
copy from hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
copy to hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
index 80d358b..18871be 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
@@ -15,31 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.client;
+package org.apache.hadoop.hbase.exceptions;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
+import java.util.Set;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * Factory class to get the instance of configured connection registry.
+ * Exception thrown when an master registry RPC fails in client. The exception includes the list of
+ * masters to which RPC was attempted and the last exception encountered. Prior exceptions are
+ * included in the logs.
  */
 @InterfaceAudience.Private
-final class ConnectionRegistryFactory {
-
-  static final String CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY =
-      "hbase.client.connection.registry.impl";
-
-  private ConnectionRegistryFactory() {
-  }
-
-  /**
-   * @return The connection registry implementation to use.
-   */
-  static ConnectionRegistry getRegistry(Configuration conf) {
-    Class<? extends ConnectionRegistry> clazz = conf.getClass(
-        CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class,
-        ConnectionRegistry.class);
-    return ReflectionUtils.newInstance(clazz, conf);
+public class MasterRegistryFetchException extends HBaseIOException {
+  public MasterRegistryFetchException(Set<ServerName> masters, Throwable failure) {
+    super(String.format("Exception making rpc to masters %s", PrettyPrinter.toString(masters)),
+        failure);
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index 629efe6..72b9f83 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -20,22 +20,6 @@ package org.apache.hadoop.hbase.ipc;
 
 import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.wrapException;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
-import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
-import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
-import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
-import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
-import org.apache.hbase.thirdparty.com.google.protobuf.Message;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
@@ -43,18 +27,15 @@ import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.codec.KeyValueCodec;
@@ -69,7 +50,22 @@ import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenSelector;
-
+import org.apache.yetus.audience.InterfaceAudience;
+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.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
+import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
+import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 
 /**
@@ -106,7 +102,8 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
 
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_MUTABLE_COLLECTION_PKGPROTECT",
       justification="the rest of the system which live in the different package can use")
-  protected final static Map<Kind, TokenSelector<? extends TokenIdentifier>> TOKEN_HANDLERS = new HashMap<>();
+  protected final static Map<Kind, TokenSelector<? extends TokenIdentifier>> TOKEN_HANDLERS =
+      new HashMap<>();
 
   static {
     TOKEN_HANDLERS.put(Kind.HBASE_AUTH_TOKEN, new AuthenticationTokenSelector());
@@ -217,7 +214,9 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
         // have some pending calls on connection so we should not shutdown the connection outside.
         // The connection itself will disconnect if there is no pending call for maxIdleTime.
         if (conn.getLastTouched() < closeBeforeTime && !conn.isActive()) {
-          if (LOG.isTraceEnabled()) LOG.trace("Cleanup idle connection to " + conn.remoteId().address);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Cleanup idle connection to {}", conn.remoteId().address);
+          }
           connections.removeValue(conn.remoteId(), conn);
           conn.cleanupConnection();
         }
@@ -398,7 +397,7 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
     }
   }
 
-  private void callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc,
+  Call callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc,
       final Message param, Message returnType, final User ticket, final InetSocketAddress addr,
       final RpcCallback<Message> callback) {
     final MetricsConnection.CallStats cs = MetricsConnection.newCallStats();
@@ -435,9 +434,10 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
     } catch (Exception e) {
       call.setException(toIOE(e));
     }
+    return call;
   }
 
-  private InetSocketAddress createAddr(ServerName sn) throws UnknownHostException {
+  InetSocketAddress createAddr(ServerName sn) throws UnknownHostException {
     InetSocketAddress addr = new InetSocketAddress(sn.getHostname(), sn.getPort());
     if (addr.isUnresolved()) {
       throw new UnknownHostException("can not resolve " + sn.getServerName());
@@ -527,6 +527,13 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
     return new RpcChannelImplementation(this, createAddr(sn), user, rpcTimeout);
   }
 
+  @Override
+  public RpcChannel createHedgedRpcChannel(Set<ServerName> sns, User user, int rpcTimeout)
+      throws UnknownHostException {
+    // Check HedgedRpcChannel implementation for detailed comments.
+    throw new UnsupportedOperationException("Hedging not supported for this implementation.");
+  }
+
   private static class AbstractRpcChannel {
 
     protected final InetSocketAddress addr;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
index f84c308..22eca53 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
@@ -17,18 +17,15 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 import java.io.IOException;
 import java.net.SocketAddress;
-
 import javax.net.SocketFactory;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
  * Does RPC against a cluster. Manages connections per regionserver in the cluster.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
new file mode 100644
index 0000000..7b681e0
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
@@ -0,0 +1,274 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first. Size of each is
+ * configurable and is also known as 'fanOutSize'.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  /**
+   * Currently hedging is only supported for non-blocking connection implementation types because
+   * the channel implementation inherently relies on the connection implementation being async.
+   * Refer to the comments in doCallMethod().
+   */
+  private final NettyRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (rpcController.failed()) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          final String exceptionMsg = String.format("%s canceled because another hedged attempt " +
+              "for the same rpc already succeeded. This is not needed anymore.", call);
+          call.setException(new CallCancelledException(exceptionMsg));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(NettyRpcClient rpcClient, Set<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = new ArrayList<>(Preconditions.checkNotNull(addrs));
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    // fanOutSize controls the number of hedged RPCs per batch.
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  private void doCallMethod(Descriptors.MethodDescriptor method, HBaseRpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // Entire batch has failed, lets try the next batch.
+      LOG.debug("Failed request {}, {}.", method.getName(), batchRpcCtx);
+      i = batchEnd;
+    }
+    Preconditions.checkNotNull(lastBatchCtx);
+    // All the batches failed, mark it a failed rpc.
+    // Propagate the failure reason. We propagate the last batch's last failing rpc reason.
+    // Can we do something better?
+    controller.setFailed(lastBatchCtx.getLastFailedRpcReason());
+    done.run(null);
+  }
+
+  @Override
+  public void callMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    // To make the channel non-blocking, we run the actual doCalMethod() async. The call back is
+    // called once the hedging finishes.
+    CompletableFuture.runAsync(
+      () -> doCallMethod(method, (HBaseRpcController)controller, request, responsePrototype, done));
+  }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
index 61dedbb..c4f70b0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
@@ -17,21 +17,26 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import org.apache.hbase.thirdparty.io.netty.channel.Channel;
-import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
-import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
-
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.SocketAddress;
-
+import java.net.UnknownHostException;
+import java.util.HashSet;
+import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.MetricsConnection;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
 
 /**
  * Netty client for the requests and responses.
@@ -75,6 +80,19 @@ public class NettyRpcClient extends AbstractRpcClient<NettyRpcConnection> {
   }
 
   @Override
+  public RpcChannel createHedgedRpcChannel(Set<ServerName> sns, User user, int rpcTimeout)
+      throws UnknownHostException {
+    final int hedgedRpcFanOut = conf.getInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY,
+        HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT);
+    Set<InetSocketAddress> addresses = new HashSet<>();
+    for (ServerName sn: sns) {
+      addresses.add(createAddr(sn));
+    }
+    return new HedgedRpcChannel(this, addresses, user, rpcTimeout,
+        hedgedRpcFanOut);
+  }
+
+  @Override
   protected void closeInternal() {
     if (shutdownGroupWhenClose) {
       group.shutdownGracefully();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
index 0e00695..558fcee 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
@@ -17,15 +17,14 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
-
 import java.io.Closeable;
 import java.io.IOException;
-
+import java.util.Set;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
 
 /**
  * Interface for RpcClient implementations so ConnectionManager can handle it.
@@ -84,6 +83,16 @@ public interface RpcClient extends Closeable {
       throws IOException;
 
   /**
+   * Creates a channel that can hedge request to multiple underlying channels.
+   * @param sns  Set of servers for underlying channels.
+   * @param user user for the connection.
+   * @param rpcTimeout rpc timeout to use.
+   * @return A hedging rpc channel for this rpc client instance.
+   */
+  RpcChannel createHedgedRpcChannel(final Set<ServerName> sns, final User user, int rpcTimeout)
+       throws IOException;
+
+  /**
    * Interrupt the connections to the given server. This should be called if the server
    * is known as actually dead. This will not prevent current operation to be retried, and,
    * depending on their own behavior, they may retry on the same server. This can be a feature,
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java
index f02ec42..561b1f5 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java
@@ -29,6 +29,7 @@ import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.FutureUtils;
@@ -70,7 +71,7 @@ public class TestConnectionRegistryLeak {
 
   @BeforeClass
   public static void setUp() {
-    CONF.setClass(ConnectionRegistryFactory.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY,
+    CONF.setClass(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY,
         ConnectionRegistryForTest.class, ConnectionRegistry.class);
   }
 
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 132d3e0..ef525d7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -180,10 +180,17 @@ public final class HConstants {
   public static final String MASTER_INFO_PORT = "hbase.master.info.port";
 
   /** Configuration key for the list of master host:ports **/
-  public static final String MASTER_ADDRS_KEY = "hbase.master.addrs";
+  public static final String MASTER_ADDRS_KEY = "hbase.masters";
 
   public static final String MASTER_ADDRS_DEFAULT =  "localhost:" + DEFAULT_MASTER_PORT;
 
+  /** Configuration to enable hedged reads on master registry **/
+  public static final String MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY =
+      "hbase.client.master_registry.enable_hedged_reads";
+
+  /** Default value for enabling hedging reads on master registry **/
+  public static final boolean MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT = false;
+
   /** Parameter name for the master type being backup (waits for primary to go inactive). */
   public static final String MASTER_TYPE_BACKUP = "hbase.master.backup";
 
@@ -909,6 +916,12 @@ public final class HConstants {
    */
   public static final String HBASE_RPC_TIMEOUT_KEY = "hbase.rpc.timeout";
 
+  /** Configuration key that controls the fan out of requests in hedged channel implementation. **/
+  public static final String HBASE_RPCS_HEDGED_REQS_FANOUT_KEY = "hbase.rpc.hedged.fanout";
+
+  /** Default value for the fan out of hedged requests. **/
+  public static final int HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT = 2;
+
   /**
    * timeout for each read RPC
    */
@@ -940,6 +953,11 @@ public final class HConstants {
    */
   public static final long NO_SEQNUM = -1;
 
+  /**
+   * Registry implementation to be used on the client side.
+   */
+  public static final String CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY =
+      "hbase.client.registry.impl";
 
   /*
    * cluster replication constants.
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java
index 147e916..ff7064b 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java
@@ -19,9 +19,12 @@
 
 package org.apache.hadoop.hbase.util;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.exceptions.HBaseException;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -29,7 +32,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
-public class PrettyPrinter {
+public final class PrettyPrinter {
 
   private static final Logger LOG = LoggerFactory.getLogger(PrettyPrinter.class);
 
@@ -117,7 +120,7 @@ public class PrettyPrinter {
       sb.append(" DAY").append(days == 1 ? "" : "S");
     }
 
-    if (hours > 0 ) {
+    if (hours > 0) {
       sb.append(days > 0 ? " " : "");
       sb.append(hours);
       sb.append(" HOUR").append(hours == 1 ? "" : "S");
@@ -188,4 +191,18 @@ public class PrettyPrinter {
     return ttl;
   }
 
+  /**
+   * Pretty prints a collection of any type to a string. Relies on toString() implementation of the
+   * object type.
+   * @param collection collection to pretty print.
+   * @return Pretty printed string for the collection.
+   */
+  public static String toString(Collection<?> collection) {
+    List<String> stringList = new ArrayList<>();
+    for (Object o: collection) {
+      stringList.add(Objects.toString(o));
+    }
+    return "[" + String.join(",", stringList) + "]";
+  }
+
 }
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTableName.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTableName.java
index 43a384a..7e44399 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTableName.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTableName.java
@@ -51,7 +51,21 @@ public class TestTableName extends TestWatcher {
    */
   @Override
   protected void starting(Description description) {
-    tableName = TableName.valueOf(description.getMethodName());
+    tableName = TableName.valueOf(cleanUpTestName(description.getMethodName()));
+  }
+
+  /**
+   * Helper to handle parameterized method names. Unlike regular test methods, parameterized method
+   * names look like 'foo[x]'. This is problematic for tests that use this name for HBase tables.
+   * This helper strips out the parameter suffixes.
+   * @return current test method name with out parameterized suffixes.
+   */
+  private static String cleanUpTestName(String methodName) {
+    int index = methodName.indexOf('[');
+    if (index == -1) {
+      return methodName;
+    }
+    return methodName.substring(0, index);
   }
 
   public TableName getTableName() {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
index 5c6ad95..cfa6f75 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
@@ -26,12 +26,14 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 /**
  * Utility used running a cluster all in the one JVM.
@@ -136,6 +138,11 @@ public class JVMClusterUtil {
     } catch (Exception e) {
       throw new IOException(e);
     }
+    // Needed if a master based registry is configured for internal cluster connections. Here, we
+    // just add the current master host port since we do not know other master addresses up front
+    // in mini cluster tests.
+    c.set(HConstants.MASTER_ADDRS_KEY,
+        Preconditions.checkNotNull(server.getServerName().getAddress()).toString());
     return new JVMClusterUtil.MasterThread(server, index);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index f1e91de..d1f2f1f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1128,6 +1128,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     this.hbaseCluster =
         new MiniHBaseCluster(c, option.getNumMasters(), option.getNumRegionServers(),
             option.getRsPorts(), option.getMasterClass(), option.getRsClass());
+    // Populate the master address configuration from mini cluster configuration.
+    conf.set(HConstants.MASTER_ADDRS_KEY,
+        c.get(HConstants.MASTER_ADDRS_KEY, HConstants.MASTER_ADDRS_DEFAULT));
     // Don't leave here till we've done a successful scan of the hbase:meta
     Table t = getConnection().getTable(TableName.META_TABLE_NAME);
     ResultScanner s = t.getScanner(new Scan());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java
index a669362..c9d67f4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 
@@ -28,7 +29,7 @@ import org.apache.hadoop.hbase.ServerName;
 public class DummyConnectionRegistry implements ConnectionRegistry {
 
   public static final String REGISTRY_IMPL_CONF_KEY =
-      ConnectionRegistryFactory.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY;
+      HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY;
 
   @Override
   public CompletableFuture<RegionLocations> getMetaRegionLocations() {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 61d78ce..4f4fe79 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -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
@@ -28,10 +28,10 @@ import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -64,7 +64,9 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TestTableName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@@ -106,23 +108,28 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.NonRepeatedEnvironmentEdge;
 import org.apache.hadoop.hbase.util.TableDescriptorChecker;
 import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.Assume;
 import org.junit.ClassRule;
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 /**
  * Run tests that use the HBase clients; {@link Table}.
  * Sets up the HBase mini cluster once at start and runs through all client tests.
  * Each creates a table named for the method and does its stuff against that.
+ *
+ * Parameterized to run with different registry implementations.
  */
 @Category({LargeTests.class, ClientTests.class})
 @SuppressWarnings ("deprecation")
+@RunWith(Parameterized.class)
 public class TestFromClientSide {
 
   @ClassRule
@@ -131,7 +138,7 @@ public class TestFromClientSide {
 
   // NOTE: Increment tests were moved to their own class, TestIncrementsFromClientSide.
   private static final Logger LOG = LoggerFactory.getLogger(TestFromClientSide.class);
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static HBaseTestingUtility TEST_UTIL;
   private static byte [] ROW = Bytes.toBytes("testRow");
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   private static final byte[] INVALID_FAMILY = Bytes.toBytes("invalidTestFamily");
@@ -139,10 +146,54 @@ public class TestFromClientSide {
   private static byte [] VALUE = Bytes.toBytes("testValue");
   protected static int SLAVES = 3;
 
-  @Rule
-  public TestName name = new TestName();
+  @Rule public TestTableName name = new TestTableName();
+
+  // To keep the child classes happy.
+  TestFromClientSide() {}
+
+  public TestFromClientSide(Class registry, int numHedgedReqs) throws Exception {
+    initialize(registry, numHedgedReqs, MultiRowMutationEndpoint.class);
+  }
+
+  @Parameterized.Parameters
+  public static Collection parameters() {
+    return Arrays.asList(new Object[][] {
+        { MasterRegistry.class, 1},
+        { MasterRegistry.class, 2},
+        { ZKConnectionRegistry.class, 1}
+    });
+  }
+
+  /**
+   * JUnit does not provide an easy way to run a hook after each parameterized run. Without that
+   * there is no easy way to restart the test cluster after each parameterized run. Annotation
+   * BeforeParam does not work either because it runs before parameterization and hence does not
+   * have access to the test parameters (which is weird).
+   *
+   * This *hack* checks if the current instance of test cluster configuration has the passed
+   * parameterized configs. In such a case, we can just reuse the cluster for test and do not need
+   * to initialize from scratch. While this is a hack, it saves a ton of time for the full
+   * test and de-flakes it.
+   */
+  private static boolean isSameParameterizedCluster(Class registryImpl, int numHedgedReqs) {
+    if (TEST_UTIL == null) {
+      return false;
+    }
+    Configuration conf = TEST_UTIL.getConfiguration();
+    Class confClass = conf.getClass(
+        HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class);
+    int hedgedReqConfig = conf.getInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY,
+        HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT);
+    return confClass.getName().equals(registryImpl.getName()) && numHedgedReqs == hedgedReqConfig;
+  }
 
-  protected static final void initialize(Class<?>... cps) throws Exception {
+  protected static final void initialize(Class registryImpl, int numHedgedReqs, Class<?>... cps)
+      throws Exception {
+    // initialize() is called for every unit test, however we only want to reset the cluster state
+    // at the end of every parameterized run.
+    if (isSameParameterizedCluster(registryImpl, numHedgedReqs)) {
+      return;
+    }
     // Uncomment the following lines if more verbosity is needed for
     // debugging (see HBASE-12285 for details).
     // ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
@@ -150,22 +201,35 @@ public class TestFromClientSide {
     // ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
     // make sure that we do not get the same ts twice, see HBASE-19731 for more details.
     EnvironmentEdgeManager.injectEdge(new NonRepeatedEnvironmentEdge());
+    if (TEST_UTIL != null) {
+      // We reached end of a parameterized run, clean up.
+      TEST_UTIL.shutdownMiniCluster();
+    }
+    TEST_UTIL = new HBaseTestingUtility();
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
       Arrays.stream(cps).map(Class::getName).toArray(String[]::new));
     conf.setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, true); // enable for below tests
-    // We need more than one region server in this test
-    TEST_UTIL.startMiniCluster(SLAVES);
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    initialize(MultiRowMutationEndpoint.class);
+    conf.setClass(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, registryImpl,
+        ConnectionRegistry.class);
+    if (numHedgedReqs == 1) {
+      conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, false);
+    } else {
+      Preconditions.checkArgument(numHedgedReqs > 1);
+      conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, true);
+    }
+    conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, numHedgedReqs);
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    // Multiple masters needed only when hedged reads for master registry are enabled.
+    builder.numMasters(numHedgedReqs > 1 ? 3 : 1).numRegionServers(SLAVES);
+    TEST_UTIL.startMiniCluster(builder.build());
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
+    if (TEST_UTIL != null) {
+      TEST_UTIL.shutdownMiniCluster();
+    }
   }
 
   /**
@@ -173,7 +237,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testDuplicateAppend() throws Exception {
-    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(name.getMethodName());
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(name.getTableName());
     Map<String, String> kvs = new HashMap<>();
     kvs.put(SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
     hdt.addCoprocessor(SleepAtFirstRpcCall.class.getName(), null, 1, kvs);
@@ -181,11 +245,11 @@ public class TestFromClientSide {
 
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
     c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
-    // Client will retry beacuse rpc timeout is small than the sleep time of first rpc call
+    // Client will retry because rpc timeout is small than the sleep time of first rpc call
     c.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 1500);
 
     try (Connection connection = ConnectionFactory.createConnection(c);
-        Table table = connection.getTableBuilder(TableName.valueOf(name.getMethodName()), null)
+        Table table = connection.getTableBuilder(name.getTableName(), null)
           .setOperationTimeout(3 * 1000).build()) {
       Append append = new Append(ROW);
       append.addColumn(HBaseTestingUtility.fam1, QUALIFIER, VALUE);
@@ -209,7 +273,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testKeepDeletedCells() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     final byte[] FAMILY = Bytes.toBytes("family");
     final byte[] C0 = Bytes.toBytes("c0");
 
@@ -275,7 +339,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testPurgeFutureDeletes() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     final byte[] ROW = Bytes.toBytes("row");
     final byte[] FAMILY = Bytes.toBytes("family");
     final byte[] COLUMN = Bytes.toBytes("column");
@@ -328,7 +392,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testGetConfiguration() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
     Configuration conf = TEST_UTIL.getConfiguration();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILIES)) {
@@ -342,7 +406,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testWeirdCacheBehaviour() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
         Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
         Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
@@ -383,7 +447,7 @@ public class TestFromClientSide {
   }
 
   private void deleteColumns(Table ht, String value, String keyPrefix)
-  throws IOException {
+      throws IOException {
     ResultScanner scanner = buildScanner(keyPrefix, value, ht);
     Iterator<Result> it = scanner.iterator();
     int count = 0;
@@ -468,8 +532,8 @@ public class TestFromClientSide {
    */
   @Test
   public void testFilterAcrossMultipleRegions()
-  throws IOException, InterruptedException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+      throws IOException, InterruptedException {
+    final TableName tableName = name.getTableName();
     try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) {
       int rowCount = TEST_UTIL.loadTable(t, FAMILY, false);
       assertRowCount(t, rowCount);
@@ -555,8 +619,7 @@ public class TestFromClientSide {
    * @param t Table to split.
    * @return Map of regions to servers.
    */
-  private List<HRegionLocation> splitTable(final Table t)
-  throws IOException, InterruptedException {
+  private List<HRegionLocation> splitTable(final Table t) throws IOException {
     // Split this table in two.
     Admin admin = TEST_UTIL.getAdmin();
     admin.split(t.getName());
@@ -572,8 +635,7 @@ public class TestFromClientSide {
    * @param t
    * @return Map of table regions; caller needs to check table actually split.
    */
-  private List<HRegionLocation> waitOnSplit(final Table t)
-  throws IOException {
+  private List<HRegionLocation> waitOnSplit(final Table t) throws IOException {
     try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName())) {
       List<HRegionLocation> regions = locator.getAllRegionLocations();
       int originalCount = regions.size();
@@ -585,8 +647,9 @@ public class TestFromClientSide {
           e.printStackTrace();
         }
         regions = locator.getAllRegionLocations();
-        if (regions.size() > originalCount)
+        if (regions.size() > originalCount) {
           break;
+        }
       }
       return regions;
     }
@@ -594,7 +657,7 @@ public class TestFromClientSide {
 
   @Test
   public void testSuperSimple() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, VALUE);
@@ -610,7 +673,7 @@ public class TestFromClientSide {
 
   @Test
   public void testMaxKeyValueSize() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     Configuration conf = TEST_UTIL.getConfiguration();
     String oldMaxSize = conf.get(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY);
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
@@ -640,7 +703,7 @@ public class TestFromClientSide {
 
   @Test
   public void testFilters() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 10);
       byte[][] QUALIFIERS = {
@@ -677,7 +740,7 @@ public class TestFromClientSide {
 
   @Test
   public void testFilterWithLongCompartor() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 10);
       byte[][] values = new byte[10][];
@@ -709,7 +772,7 @@ public class TestFromClientSide {
 
   @Test
   public void testKeyOnlyFilter() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 10);
       byte[][] QUALIFIERS = {
@@ -747,7 +810,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testSimpleMissing() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 4);
 
@@ -858,7 +921,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testSingleRowMultipleFamily() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] ROWS = makeN(ROW, 3);
     byte [][] FAMILIES = makeNAscii(FAMILY, 10);
     byte [][] QUALIFIERS = makeN(QUALIFIER, 10);
@@ -1166,7 +1229,7 @@ public class TestFromClientSide {
 
   @Test(expected = IllegalArgumentException.class)
   public void testNullFamilyName() throws IOException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     // Null family (should NOT work)
     TEST_UTIL.createTable(tableName, new byte[][]{null});
@@ -1175,7 +1238,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNullRowAndQualifier() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
 
@@ -1211,7 +1274,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNullEmptyQualifier() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
 
@@ -1249,7 +1312,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNullValue() throws IOException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       // Null value
@@ -1284,7 +1347,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNullQualifier() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
 
       // Work for Put
@@ -1342,7 +1405,7 @@ public class TestFromClientSide {
 
   @Test
   public void testVersions() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     long [] STAMPS = makeStamps(20);
     byte [][] VALUES = makeNAscii(VALUE, 20);
@@ -1569,7 +1632,7 @@ public class TestFromClientSide {
   @Test
   @SuppressWarnings("checkstyle:MethodLength")
   public void testVersionLimits() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
     int [] LIMITS = {1,3,5};
     long [] STAMPS = makeStamps(10);
@@ -1764,7 +1827,7 @@ public class TestFromClientSide {
   @Test
   public void testDeleteFamilyVersion() throws Exception {
     try (Admin admin = TEST_UTIL.getAdmin()) {
-      final TableName tableName = TableName.valueOf(name.getMethodName());
+      final TableName tableName = name.getTableName();
 
       byte[][] QUALIFIERS = makeNAscii(QUALIFIER, 1);
       byte[][] VALUES = makeN(VALUE, 5);
@@ -1804,7 +1867,7 @@ public class TestFromClientSide {
 
   @Test
   public void testDeleteFamilyVersionWithOtherDeletes() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 5);
     byte [][] VALUES = makeN(VALUE, 5);
@@ -1922,7 +1985,7 @@ public class TestFromClientSide {
 
   @Test
   public void testDeleteWithFailed() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
     byte [][] VALUES = makeN(VALUE, 5);
@@ -1948,7 +2011,7 @@ public class TestFromClientSide {
 
   @Test
   public void testDeletes() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     byte [][] ROWS = makeNAscii(ROW, 6);
     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
@@ -2254,7 +2317,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testBatchOperationsWithErrors() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table foo = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, 10)) {
 
       int NUM_OPS = 100;
@@ -2380,7 +2443,7 @@ public class TestFromClientSide {
     int numRows = 10;
     int numColsPerRow = 2000;
 
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     byte [][] ROWS = makeN(ROW, numRows);
     byte [][] QUALIFIERS = makeN(QUALIFIER, numColsPerRow);
@@ -2463,7 +2526,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testJiraTest861() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] VALUES = makeNAscii(VALUE, 7);
     long [] STAMPS = makeStamps(7);
 
@@ -2526,7 +2589,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testJiraTest33() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] VALUES = makeNAscii(VALUE, 7);
     long [] STAMPS = makeStamps(7);
 
@@ -2574,7 +2637,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testJiraTest1014() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 10)) {
 
@@ -2598,7 +2661,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testJiraTest1182() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] VALUES = makeNAscii(VALUE, 7);
     long [] STAMPS = makeStamps(7);
 
@@ -2642,7 +2705,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testJiraTest52() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] VALUES = makeNAscii(VALUE, 7);
     long [] STAMPS = makeStamps(7);
 
@@ -2678,8 +2741,7 @@ public class TestFromClientSide {
 
   private void getVersionRangeAndVerifyGreaterThan(Table ht, byte [] row,
       byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
-      int start, int end)
-  throws IOException {
+      int start, int end) throws IOException {
     Get get = new Get(row);
     get.addColumn(family, qualifier);
     get.readVersions(Integer.MAX_VALUE);
@@ -2689,8 +2751,7 @@ public class TestFromClientSide {
   }
 
   private void getVersionRangeAndVerify(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
-  throws IOException {
+      byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException {
     Get get = new Get(row);
     get.addColumn(family, qualifier);
     get.readVersions(Integer.MAX_VALUE);
@@ -2700,8 +2761,7 @@ public class TestFromClientSide {
   }
 
   private void getAllVersionsAndVerify(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
-  throws IOException {
+      byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException {
     Get get = new Get(row);
     get.addColumn(family, qualifier);
     get.readVersions(Integer.MAX_VALUE);
@@ -2711,8 +2771,7 @@ public class TestFromClientSide {
 
   private void scanVersionRangeAndVerifyGreaterThan(Table ht, byte [] row,
       byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
-      int start, int end)
-  throws IOException {
+      int start, int end) throws IOException {
     Scan scan = new Scan(row);
     scan.addColumn(family, qualifier);
     scan.setMaxVersions(Integer.MAX_VALUE);
@@ -2722,8 +2781,7 @@ public class TestFromClientSide {
   }
 
   private void scanVersionRangeAndVerify(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
-  throws IOException {
+      byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException {
     Scan scan = new Scan(row);
     scan.addColumn(family, qualifier);
     scan.setMaxVersions(Integer.MAX_VALUE);
@@ -2733,8 +2791,7 @@ public class TestFromClientSide {
   }
 
   private void scanAllVersionsAndVerify(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
-  throws IOException {
+      byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException {
     Scan scan = new Scan(row);
     scan.addColumn(family, qualifier);
     scan.setMaxVersions(Integer.MAX_VALUE);
@@ -2743,8 +2800,7 @@ public class TestFromClientSide {
   }
 
   private void getVersionAndVerify(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long stamp, byte [] value)
-  throws Exception {
+      byte [] qualifier, long stamp, byte [] value) throws Exception {
     Get get = new Get(row);
     get.addColumn(family, qualifier);
     get.setTimestamp(stamp);
@@ -2754,8 +2810,7 @@ public class TestFromClientSide {
   }
 
   private void getVersionAndVerifyMissing(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long stamp)
-  throws Exception {
+      byte [] qualifier, long stamp) throws Exception {
     Get get = new Get(row);
     get.addColumn(family, qualifier);
     get.setTimestamp(stamp);
@@ -2765,8 +2820,7 @@ public class TestFromClientSide {
   }
 
   private void scanVersionAndVerify(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long stamp, byte [] value)
-  throws Exception {
+      byte [] qualifier, long stamp, byte [] value) throws Exception {
     Scan scan = new Scan(row);
     scan.addColumn(family, qualifier);
     scan.setTimestamp(stamp);
@@ -2776,8 +2830,7 @@ public class TestFromClientSide {
   }
 
   private void scanVersionAndVerifyMissing(Table ht, byte [] row,
-      byte [] family, byte [] qualifier, long stamp)
-  throws Exception {
+      byte [] family, byte [] qualifier, long stamp) throws Exception {
     Scan scan = new Scan(row);
     scan.addColumn(family, qualifier);
     scan.setTimestamp(stamp);
@@ -2786,10 +2839,7 @@ public class TestFromClientSide {
     assertNullResult(result);
   }
 
-  private void getTestNull(Table ht, byte [] row, byte [] family,
-      byte [] value)
-  throws Exception {
-
+  private void getTestNull(Table ht, byte [] row, byte [] family, byte [] value) throws Exception {
     Get get = new Get(row);
     get.addColumn(family, null);
     Result result = ht.get(get);
@@ -2866,9 +2916,7 @@ public class TestFromClientSide {
   }
 
   private void singleRowGetTest(Table ht, byte [][] ROWS, byte [][] FAMILIES,
-      byte [][] QUALIFIERS, byte [][] VALUES)
-  throws Exception {
-
+      byte [][] QUALIFIERS, byte [][] VALUES) throws Exception {
     // Single column from memstore
     Get get = new Get(ROWS[0]);
     get.addColumn(FAMILIES[4], QUALIFIERS[0]);
@@ -2923,7 +2971,7 @@ public class TestFromClientSide {
     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
         new int [][] {
           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
-    });
+        });
 
     // Multiple columns from everywhere storefile, many family, wildcard
     get = new Get(ROWS[0]);
@@ -2939,7 +2987,7 @@ public class TestFromClientSide {
     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
         new int [][] {
           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
-    });
+        });
 
     // Everything
     get = new Get(ROWS[0]);
@@ -2947,7 +2995,7 @@ public class TestFromClientSide {
     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
         new int [][] {
           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0}
-    });
+        });
 
     // Get around inserted columns
 
@@ -2964,9 +3012,7 @@ public class TestFromClientSide {
   }
 
   private void singleRowScanTest(Table ht, byte [][] ROWS, byte [][] FAMILIES,
-      byte [][] QUALIFIERS, byte [][] VALUES)
-  throws Exception {
-
+      byte [][] QUALIFIERS, byte [][] VALUES) throws Exception {
     // Single column from memstore
     Scan scan = new Scan();
     scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
@@ -3021,7 +3067,7 @@ public class TestFromClientSide {
     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
         new int [][] {
           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
-    });
+        });
 
     // Multiple columns from everywhere storefile, many family, wildcard
     scan = new Scan();
@@ -3037,7 +3083,7 @@ public class TestFromClientSide {
     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
         new int [][] {
           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
-    });
+        });
 
     // Everything
     scan = new Scan();
@@ -3045,7 +3091,7 @@ public class TestFromClientSide {
     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
         new int [][] {
           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0}
-    });
+        });
 
     // Scan around inserted columns
 
@@ -3065,13 +3111,9 @@ public class TestFromClientSide {
    * Expects family and qualifier arrays to be valid for at least
    * the range:  idx-2 < idx < idx+2
    */
-  private void getVerifySingleColumn(Table ht,
-      byte [][] ROWS, int ROWIDX,
-      byte [][] FAMILIES, int FAMILYIDX,
-      byte [][] QUALIFIERS, int QUALIFIERIDX,
-      byte [][] VALUES, int VALUEIDX)
-  throws Exception {
-
+  private void getVerifySingleColumn(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES,
+      int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX, byte [][] VALUES, int VALUEIDX)
+      throws Exception {
     Get get = new Get(ROWS[ROWIDX]);
     Result result = ht.get(get);
     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
@@ -3123,13 +3165,9 @@ public class TestFromClientSide {
    * the range:  idx-2 to idx+2
    * Expects row array to be valid for at least idx to idx+2
    */
-  private void scanVerifySingleColumn(Table ht,
-      byte [][] ROWS, int ROWIDX,
-      byte [][] FAMILIES, int FAMILYIDX,
-      byte [][] QUALIFIERS, int QUALIFIERIDX,
-      byte [][] VALUES, int VALUEIDX)
-  throws Exception {
-
+  private void scanVerifySingleColumn(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES,
+      int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX, byte [][] VALUES, int VALUEIDX)
+      throws Exception {
     Scan scan = new Scan();
     Result result = getSingleScanResult(ht, scan);
     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
@@ -3183,12 +3221,8 @@ public class TestFromClientSide {
    * Verify we do not read any values by accident around a single column
    * Same requirements as getVerifySingleColumn
    */
-  private void getVerifySingleEmpty(Table ht,
-      byte [][] ROWS, int ROWIDX,
-      byte [][] FAMILIES, int FAMILYIDX,
-      byte [][] QUALIFIERS, int QUALIFIERIDX)
-  throws Exception {
-
+  private void getVerifySingleEmpty(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES,
+      int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX) throws Exception {
     Get get = new Get(ROWS[ROWIDX]);
     get.addFamily(FAMILIES[4]);
     get.addColumn(FAMILIES[4], QUALIFIERS[1]);
@@ -3214,12 +3248,8 @@ public class TestFromClientSide {
 
   }
 
-  private void scanVerifySingleEmpty(Table ht,
-      byte [][] ROWS, int ROWIDX,
-      byte [][] FAMILIES, int FAMILYIDX,
-      byte [][] QUALIFIERS, int QUALIFIERIDX)
-  throws Exception {
-
+  private void scanVerifySingleEmpty(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES,
+      int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX) throws Exception {
     Scan scan = new Scan(ROWS[ROWIDX+1]);
     Result result = getSingleScanResult(ht, scan);
     assertNullResult(result);
@@ -3244,9 +3274,7 @@ public class TestFromClientSide {
   // Verifiers
   //
 
-  private void assertKey(Cell key, byte [] row, byte [] family,
-      byte [] qualifier, byte [] value)
-  throws Exception {
+  private void assertKey(Cell key, byte [] row, byte [] family, byte [] qualifier, byte [] value) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]",
         equals(row, CellUtil.cloneRow(key)));
@@ -3262,8 +3290,7 @@ public class TestFromClientSide {
   }
 
   static void assertIncrementKey(Cell key, byte [] row, byte [] family,
-      byte [] qualifier, long value)
-  throws Exception {
+      byte [] qualifier, long value) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]",
         equals(row, CellUtil.cloneRow(key)));
@@ -3284,9 +3311,7 @@ public class TestFromClientSide {
   }
 
   private void assertNResult(Result result, byte [] row,
-      byte [][] families, byte [][] qualifiers, byte [][] values,
-      int [][] idxs)
-  throws Exception {
+      byte [][] families, byte [][] qualifiers, byte [][] values, int [][] idxs) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(result.getRow()) +"]",
         equals(row, result.getRow()));
@@ -3318,8 +3343,7 @@ public class TestFromClientSide {
 
   private void assertNResult(Result result, byte [] row,
       byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
-      int start, int end)
-  throws IOException {
+      int start, int end) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(result.getRow()) +"]",
         equals(row, result.getRow()));
@@ -3353,8 +3377,7 @@ public class TestFromClientSide {
    */
   private void assertDoubleResult(Result result, byte [] row,
       byte [] familyA, byte [] qualifierA, byte [] valueA,
-      byte [] familyB, byte [] qualifierB, byte [] valueB)
-  throws Exception {
+      byte [] familyB, byte [] qualifierB, byte [] valueB) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(result.getRow()) +"]",
         equals(row, result.getRow()));
@@ -3384,8 +3407,7 @@ public class TestFromClientSide {
   }
 
   private void assertSingleResult(Result result, byte [] row, byte [] family,
-      byte [] qualifier, byte [] value)
-  throws Exception {
+      byte [] qualifier, byte [] value) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(result.getRow()) +"]",
         equals(row, result.getRow()));
@@ -3423,8 +3445,7 @@ public class TestFromClientSide {
   }
 
   private void assertSingleResult(Result result, byte [] row, byte [] family,
-      byte [] qualifier, long ts, byte [] value)
-  throws Exception {
+      byte [] qualifier, long ts, byte [] value) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(result.getRow()) +"]",
         equals(row, result.getRow()));
@@ -3515,7 +3536,7 @@ public class TestFromClientSide {
 
   @Test
   public void testDuplicateVersions() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     long [] STAMPS = makeStamps(20);
     byte [][] VALUES = makeNAscii(VALUE, 20);
@@ -3738,7 +3759,7 @@ public class TestFromClientSide {
 
   @Test
   public void testUpdates() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10)) {
 
       // Write a column with values at timestamp 1, 2 and 3
@@ -3788,7 +3809,7 @@ public class TestFromClientSide {
 
   @Test
   public void testUpdatesWithMajorCompaction() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10);
         Admin admin = TEST_UTIL.getAdmin()) {
 
@@ -3849,7 +3870,7 @@ public class TestFromClientSide {
 
   @Test
   public void testMajorCompactionBetweenTwoUpdates() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10);
         Admin admin = TEST_UTIL.getAdmin()) {
 
@@ -3916,7 +3937,7 @@ public class TestFromClientSide {
 
   @Test
   public void testGet_EmptyTable() throws IOException {
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       Get get = new Get(ROW);
       get.addFamily(FAMILY);
       Result r = table.get(get);
@@ -3926,7 +3947,7 @@ public class TestFromClientSide {
 
   @Test
   public void testGet_NullQualifier() throws IOException {
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, VALUE);
       table.put(put);
@@ -3950,7 +3971,7 @@ public class TestFromClientSide {
 
   @Test
   public void testGet_NonExistentRow() throws IOException {
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, VALUE);
       table.put(put);
@@ -3978,7 +3999,7 @@ public class TestFromClientSide {
     final byte [] row1 = Bytes.toBytes("row1");
     final byte [] row2 = Bytes.toBytes("row2");
     final byte [] value = Bytes.toBytes("abcd");
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
+    try (Table table = TEST_UTIL.createTable(name.getTableName(),
         new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY })) {
       Put put = new Put(row1);
       put.addColumn(CONTENTS_FAMILY, null, value);
@@ -4017,7 +4038,7 @@ public class TestFromClientSide {
   public void testPutNoCF() throws IOException {
     final byte[] BAD_FAM = Bytes.toBytes("BAD_CF");
     final byte[] VAL = Bytes.toBytes(100);
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       boolean caughtNSCFE = false;
 
       try {
@@ -4037,7 +4058,7 @@ public class TestFromClientSide {
     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
     final int NB_BATCH_ROWS = 10;
     final byte[] value = Bytes.toBytes("abcd");
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
+    try (Table table = TEST_UTIL.createTable(name.getTableName(),
       new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY })) {
       ArrayList<Put> rowsUpdate = new ArrayList<Put>();
       for (int i = 0; i < NB_BATCH_ROWS; i++) {
@@ -4067,7 +4088,7 @@ public class TestFromClientSide {
     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
     final byte[] value = Bytes.toBytes("abcd");
     final int NB_BATCH_ROWS = 10;
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
+    try (Table table = TEST_UTIL.createTable(name.getTableName(),
         new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY })) {
       ArrayList<Put> rowsUpdate = new ArrayList<Put>();
       for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
@@ -4130,7 +4151,7 @@ public class TestFromClientSide {
     final byte [] FAM1 = Bytes.toBytes("fam1");
     final byte [] FAM2 = Bytes.toBytes("fam2");
     // Open table
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
+    try (Table table = TEST_UTIL.createTable(name.getTableName(),
       new byte [][] {FAM1, FAM2})) {
       // Insert some values
       Put put = new Put(ROW);
@@ -4213,9 +4234,10 @@ public class TestFromClientSide {
 
   @Test
   public void testListTables() throws IOException, InterruptedException {
-    final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1");
-    final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2");
-    final TableName tableName3 = TableName.valueOf(name.getMethodName() + "3");
+    final String testTableName = name.getTableName().toString();
+    final TableName tableName1 = TableName.valueOf(testTableName + "1");
+    final TableName tableName2 = TableName.valueOf(testTableName + "2");
+    final TableName tableName3 = TableName.valueOf(testTableName + "3");
     TableName [] tables = new TableName[] { tableName1, tableName2, tableName3 };
     for (int i = 0; i < tables.length; i++) {
       TEST_UTIL.createTable(tables[i], FAMILY);
@@ -4244,7 +4266,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testUnmanagedHConnection() throws IOException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
     try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
         Table t = conn.getTable(tableName);
@@ -4260,7 +4282,13 @@ public class TestFromClientSide {
    */
   @Test
   public void testUnmanagedHConnectionReconnect() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    Configuration conf = TEST_UTIL.getConfiguration();
+    Class registryImpl = conf.getClass(
+        HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class);
+    // This test does not make sense for MasterRegistry since it stops the only master in the
+    // cluster and starts a new master without populating the underlying config for the connection.
+    Assume.assumeFalse(registryImpl.equals(MasterRegistry.class));
+    final TableName tableName = name.getTableName();
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
     try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
       try (Table t = conn.getTable(tableName); Admin admin = conn.getAdmin()) {
@@ -4290,8 +4318,9 @@ public class TestFromClientSide {
 
   @Test
   public void testMiscHTableStuff() throws IOException {
-    final TableName tableAname = TableName.valueOf(name.getMethodName() + "A");
-    final TableName tableBname = TableName.valueOf(name.getMethodName() + "B");
+    final String testTableName = name.getTableName().toString();
+    final TableName tableAname = TableName.valueOf(testTableName + "A");
+    final TableName tableBname = TableName.valueOf(testTableName + "B");
     final byte[] attrName = Bytes.toBytes("TESTATTR");
     final byte[] attrValue = Bytes.toBytes("somevalue");
     byte[] value = Bytes.toBytes("value");
@@ -4340,8 +4369,9 @@ public class TestFromClientSide {
       // add a user attribute to HTD
       desc.setValue(attrName, attrValue);
       // add a user attribute to HCD
-      for (HColumnDescriptor c : desc.getFamilies())
+      for (HColumnDescriptor c : desc.getFamilies()) {
         c.setValue(attrName, attrValue);
+      }
       // update metadata for all regions of this table
       admin.modifyTable(desc);
       // enable the table
@@ -4365,7 +4395,7 @@ public class TestFromClientSide {
 
   @Test
   public void testGetClosestRowBefore() throws IOException, InterruptedException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     final byte[] firstRow = Bytes.toBytes("row111");
     final byte[] secondRow = Bytes.toBytes("row222");
     final byte[] thirdRow = Bytes.toBytes("row333");
@@ -4492,7 +4522,7 @@ public class TestFromClientSide {
   @Test
   public void testMultiRowMutation() throws Exception {
     LOG.info("Starting testMultiRowMutation");
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     final byte [] ROW1 = Bytes.toBytes("testRow1");
 
     try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) {
@@ -4524,7 +4554,7 @@ public class TestFromClientSide {
   @Test
   public void testRowMutation() throws Exception {
     LOG.info("Starting testRowMutation");
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] QUALIFIERS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b") };
       RowMutations arm = new RowMutations(ROW);
@@ -4574,7 +4604,7 @@ public class TestFromClientSide {
   @Test
   public void testBatchAppendWithReturnResultFalse() throws Exception {
     LOG.info("Starting testBatchAppendWithReturnResultFalse");
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
       Append append1 = new Append(Bytes.toBytes("row1"));
       append1.setReturnResults(false);
@@ -4598,7 +4628,7 @@ public class TestFromClientSide {
   @Test
   public void testAppend() throws Exception {
     LOG.info("Starting testAppend");
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[] v1 = Bytes.toBytes("42");
       byte[] v2 = Bytes.toBytes("23");
@@ -4690,7 +4720,7 @@ public class TestFromClientSide {
 
   @Test
   public void testClientPoolRoundRobin() throws IOException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     int poolSize = 3;
     int numVersions = poolSize * 2;
@@ -4728,7 +4758,7 @@ public class TestFromClientSide {
 
   @Ignore ("Flakey: HBASE-8989") @Test
   public void testClientPoolThreadLocal() throws IOException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     int poolSize = Integer.MAX_VALUE;
     int numVersions = 3;
@@ -4814,7 +4844,7 @@ public class TestFromClientSide {
     final byte [] anotherrow = Bytes.toBytes("anotherrow");
     final byte [] value2 = Bytes.toBytes("abcd");
 
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       Put put1 = new Put(ROW);
       put1.addColumn(FAMILY, QUALIFIER, VALUE);
 
@@ -4852,7 +4882,7 @@ public class TestFromClientSide {
 
   @Test
   public void testCheckAndMutateWithTimeRange() throws IOException {
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       final long ts = System.currentTimeMillis() / 2;
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, ts, VALUE);
@@ -4948,7 +4978,7 @@ public class TestFromClientSide {
     final byte [] value3 = Bytes.toBytes("cccc");
     final byte [] value4 = Bytes.toBytes("dddd");
 
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
 
       Put put2 = new Put(ROW);
       put2.addColumn(FAMILY, QUALIFIER, value2);
@@ -5030,7 +5060,7 @@ public class TestFromClientSide {
   public void testCheckAndDelete() throws IOException {
     final byte [] value1 = Bytes.toBytes("aaaa");
 
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
+    try (Table table = TEST_UTIL.createTable(name.getTableName(),
         FAMILY)) {
 
       Put put = new Put(ROW);
@@ -5053,7 +5083,7 @@ public class TestFromClientSide {
     final byte [] value3 = Bytes.toBytes("cccc");
     final byte [] value4 = Bytes.toBytes("dddd");
 
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
+    try (Table table = TEST_UTIL.createTable(name.getTableName(),
         FAMILY)) {
 
       Put put2 = new Put(ROW);
@@ -5143,9 +5173,9 @@ public class TestFromClientSide {
   * Test ScanMetrics
   */
   @Test
-  @SuppressWarnings ("unused")
+  @SuppressWarnings({"unused", "checkstyle:EmptyBlock"})
   public void testScanMetrics() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     // Set up test table:
     // Create table:
@@ -5198,7 +5228,6 @@ public class TestFromClientSide {
         // the end of the scanner. So this is asking for 2 of the 3 rows we inserted.
         for (Result result : scanner.next(numRecords - 1)) {
         }
-
         ScanMetrics scanMetrics = scanner.getScanMetrics();
         assertEquals("Did not access all the regions in the table", numOfRegions,
                 scanMetrics.countOfRegions.get());
@@ -5279,7 +5308,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testCacheOnWriteEvictOnClose() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [] data = Bytes.toBytes("data");
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
       try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
@@ -5403,7 +5432,7 @@ public class TestFromClientSide {
    */
   public void testNonCachedGetRegionLocation() throws Exception {
     // Test Initialization.
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [] family1 = Bytes.toBytes("f1");
     byte [] family2 = Bytes.toBytes("f2");
     try (Table table = TEST_UTIL.createTable(tableName, new byte[][] {family1, family2}, 10);
@@ -5452,7 +5481,7 @@ public class TestFromClientSide {
     // Test Initialization.
     byte [] startKey = Bytes.toBytes("ddc");
     byte [] endKey = Bytes.toBytes("mmm");
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     TEST_UTIL.createMultiRegionTable(tableName, new byte[][] { FAMILY }, 10);
 
     int numOfRegions = -1;
@@ -5522,7 +5551,7 @@ public class TestFromClientSide {
 
   @Test
   public void testJira6912() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table foo = TEST_UTIL.createTable(tableName, new byte[][] {FAMILY}, 10)) {
 
       List<Put> puts = new ArrayList<Put>();
@@ -5551,7 +5580,7 @@ public class TestFromClientSide {
 
   @Test
   public void testScan_NullQualifier() throws IOException {
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, VALUE);
       table.put(put);
@@ -5581,7 +5610,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNegativeTimestamp() throws IOException {
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
 
       try {
         Put put = new Put(ROW, -1);
@@ -5642,7 +5671,7 @@ public class TestFromClientSide {
 
   @Test
   public void testRawScanRespectsVersions() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[] row = Bytes.toBytes("row");
 
@@ -5716,7 +5745,7 @@ public class TestFromClientSide {
   @Test
   public void testEmptyFilterList() throws Exception {
     // Test Initialization.
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
 
       // Insert one row each region
@@ -5756,7 +5785,7 @@ public class TestFromClientSide {
   @Test
   public void testSmallScan() throws Exception {
     // Test Initialization.
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
 
       // Insert one row each region
@@ -5794,7 +5823,7 @@ public class TestFromClientSide {
 
   @Test
   public void testSuperSimpleWithReverseScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000"));
       put.addColumn(FAMILY, QUALIFIER, VALUE);
@@ -5839,7 +5868,7 @@ public class TestFromClientSide {
 
   @Test
   public void testFiltersWithReverseScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 10);
       byte[][] QUALIFIERS = {Bytes.toBytes("col0-<d2v1>-<d3v2>"),
@@ -5882,7 +5911,7 @@ public class TestFromClientSide {
 
   @Test
   public void testKeyOnlyFilterWithReverseScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 10);
       byte[][] QUALIFIERS = {Bytes.toBytes("col0-<d2v1>-<d3v2>"),
@@ -5923,7 +5952,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testSimpleMissingWithReverseScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 4);
 
@@ -5988,7 +6017,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNullWithReverseScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       // Null qualifier (should work)
       Put put = new Put(ROW);
@@ -6001,7 +6030,8 @@ public class TestFromClientSide {
     }
 
     // Use a new table
-    try (Table ht = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName() + "2"), FAMILY)) {
+    try (Table ht =
+       TEST_UTIL.createTable(TableName.valueOf(name.getTableName().toString() + "2"), FAMILY)) {
       // Empty qualifier, byte[0] instead of null (should work)
       Put put = new Put(ROW);
       put.addColumn(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE);
@@ -6027,7 +6057,7 @@ public class TestFromClientSide {
   @Test
   @SuppressWarnings("checkstyle:MethodLength")
   public void testDeletesWithReverseScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte[][] ROWS = makeNAscii(ROW, 6);
     byte[][] FAMILIES = makeNAscii(FAMILY, 3);
     byte[][] VALUES = makeN(VALUE, 5);
@@ -6213,7 +6243,7 @@ public class TestFromClientSide {
   @Test
   public void testReversedScanUnderMultiRegions() throws Exception {
     // Test Initialization.
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte[] maxByteArray = ConnectionUtils.MAX_BYTE_ARRAY;
     byte[][] splitRows = new byte[][] { Bytes.toBytes("005"),
         Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)),
@@ -6274,7 +6304,7 @@ public class TestFromClientSide {
   @Test
   public void testSmallReversedScanUnderMultiRegions() throws Exception {
     // Test Initialization.
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte[][] splitRows = new byte[][]{
         Bytes.toBytes("000"), Bytes.toBytes("002"), Bytes.toBytes("004"),
         Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010")};
@@ -6494,7 +6524,7 @@ public class TestFromClientSide {
   @Test
   public void testDeleteSpecifiedVersionOfSpecifiedColumn() throws Exception {
     try (Admin admin = TEST_UTIL.getAdmin()) {
-      final TableName tableName = TableName.valueOf(name.getMethodName());
+      final TableName tableName = name.getTableName();
 
       byte[][] VALUES = makeN(VALUE, 5);
       long[] ts = {1000, 2000, 3000, 4000, 5000};
@@ -6540,7 +6570,7 @@ public class TestFromClientSide {
   @Test
   public void testDeleteLatestVersionOfSpecifiedColumn() throws Exception {
     try (Admin admin = TEST_UTIL.getAdmin()) {
-      final TableName tableName = TableName.valueOf(name.getMethodName());
+      final TableName tableName = name.getTableName();
 
       byte[][] VALUES = makeN(VALUE, 5);
       long[] ts = {1000, 2000, 3000, 4000, 5000};
@@ -6603,7 +6633,7 @@ public class TestFromClientSide {
   @Test
   public void testReadWithFilter() throws Exception {
     try (Admin admin = TEST_UTIL.getAdmin()) {
-      final TableName tableName = TableName.valueOf(name.getMethodName());
+      final TableName tableName = name.getTableName();
       try (Table table = TEST_UTIL.createTable(tableName, FAMILY, 3)) {
 
         byte[] VALUEA = Bytes.toBytes("value-a");
@@ -6690,7 +6720,7 @@ public class TestFromClientSide {
 
   @Test
   public void testCellUtilTypeMethods() throws IOException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
 
       final byte[] row = Bytes.toBytes("p");
@@ -6744,7 +6774,7 @@ public class TestFromClientSide {
 
   @Test(expected = DoNotRetryIOException.class)
   public void testCreateTableWithZeroRegionReplicas() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
         .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf")))
         .setRegionReplication(0)
@@ -6755,7 +6785,7 @@ public class TestFromClientSide {
 
   @Test(expected = DoNotRetryIOException.class)
   public void testModifyTableWithZeroRegionReplicas() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
         .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf")))
         .build();
@@ -6770,13 +6800,13 @@ public class TestFromClientSide {
 
   @Test(timeout = 60000)
   public void testModifyTableWithMemstoreData() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     createTableAndValidateTableSchemaModification(tableName, true);
   }
 
   @Test(timeout = 60000)
   public void testDeleteCFWithMemstoreData() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     createTableAndValidateTableSchemaModification(tableName, false);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
index 37d0135..8845f9a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
@@ -17,14 +17,16 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.util.Arrays;
+import java.util.Collection;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
+import org.junit.runners.Parameterized;
 
 /**
  * Test all client operations with a coprocessor that just implements the default flush/compact/scan
@@ -32,13 +34,24 @@ import org.junit.experimental.categories.Category;
  */
 @Category({ LargeTests.class, ClientTests.class })
 public class TestFromClientSideWithCoprocessor extends TestFromClientSide {
-
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
     HBaseClassTestRule.forClass(TestFromClientSideWithCoprocessor.class);
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    initialize(MultiRowMutationEndpoint.class, NoOpScanPolicyObserver.class);
+  // Override the parameters from the parent class. We just want to run it for the default
+  // param combination.
+  @Parameterized.Parameters
+  public static Collection parameters() {
+    return Arrays.asList(new Object[][] {
+        { ZKConnectionRegistry.class, 1}
+    });
+  }
+
+  public TestFromClientSideWithCoprocessor(Class registry, int numHedgedReqs) throws Exception {
+    if (TEST_UTIL == null) {
+      // It is ok to initialize once because the test is parameterized for a single dimension.
+      initialize(registry, numHedgedReqs, NoOpScanPolicyObserver.class,
+          MultiRowMutationEndpoint.class);
+    }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
new file mode 100644
index 0000000..335f968
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.apache.hadoop.hbase.HConstants.META_REPLICAS_NUM;
+import static org.junit.Assert.assertEquals;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.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.StartMiniClusterOption;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestMasterRegistry {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterRegistry.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3);
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    builder.numMasters(3).numRegionServers(3);
+    TEST_UTIL.startMiniCluster(builder.build());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Generates a string of dummy master addresses in host:port format. Every other hostname won't
+   * have a port number.
+   */
+  private static String generateDummyMastersList(int size) {
+    List<String> masters = new ArrayList<>();
+    for (int i = 0; i < size; i++) {
+      masters.add(" localhost" + (i % 2 == 0 ? ":" + (1000 + i) : ""));
+    }
+    return String.join(",", masters);
+  }
+
+  /**
+   * Makes sure the master registry parses the master end points in the configuration correctly.
+   */
+  @Test public void testMasterAddressParsing() {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    int numMasters = 10;
+    conf.set(HConstants.MASTER_ADDRS_KEY, generateDummyMastersList(numMasters));
+    try (MasterRegistry registry = new MasterRegistry(conf)) {
+      List<ServerName> parsedMasters = new ArrayList<>(registry.getParsedMasterServers());
+      // Half of them would be without a port, duplicates are removed.
+      assertEquals(numMasters/2 + 1, parsedMasters.size());
+      // Sort in the increasing order of port numbers.
+      Collections.sort(parsedMasters, Comparator.comparingInt(ServerName::getPort));
+      for (int i = 0; i < parsedMasters.size(); i++) {
+        ServerName sn = parsedMasters.get(i);
+        assertEquals("localhost", sn.getHostname());
+        if (i == parsedMasters.size() - 1) {
+          // Last entry should be the one with default port.
+          assertEquals(HConstants.DEFAULT_MASTER_PORT, sn.getPort());
+        } else {
+          assertEquals(1000 + (2 * i), sn.getPort());
+        }
+      }
+    }
+  }
+
+  @Test public void testRegistryRPCs() throws Exception {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    HMaster activeMaster = TEST_UTIL.getHBaseCluster().getMaster();
+    for (int numHedgedReqs = 1; numHedgedReqs <=3; numHedgedReqs++) {
+      if (numHedgedReqs == 1) {
+        conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, false);
+      } else {
+        conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, true);
+      }
+      conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, numHedgedReqs);
+      try (MasterRegistry registry = new MasterRegistry(conf)) {
+        assertEquals(registry.getClusterId().get(), activeMaster.getClusterId());
+        assertEquals(registry.getActiveMaster().get(), activeMaster.getServerName());
+        List<HRegionLocation> metaLocations =
+            Arrays.asList(registry.getMetaRegionLocations().get().getRegionLocations());
+        List<HRegionLocation> actualMetaLocations = activeMaster.getMetaRegionLocationCache()
+            .getMetaRegionLocations().get();
+        Collections.sort(metaLocations);
+        Collections.sort(actualMetaLocations);
+        assertEquals(actualMetaLocations, metaLocations);
+      }
+    }
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
index c3da587..796ebb3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
@@ -29,9 +29,10 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
@@ -44,35 +45,38 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTestConst;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.filter.FilterBase;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.TestTableName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
 import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
+import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 /**
- * A client-side test, mostly testing scanners with various parameters.
+ * A client-side test, mostly testing scanners with various parameters. Parameterized on different
+ * registry implementations.
  */
 @Category({MediumTests.class, ClientTests.class})
+@RunWith(Parameterized.class)
 public class TestScannersFromClientSide {
 
   @ClassRule
@@ -81,38 +85,80 @@ public class TestScannersFromClientSide {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestScannersFromClientSide.class);
 
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static HBaseTestingUtility TEST_UTIL;
   private static byte [] ROW = Bytes.toBytes("testRow");
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
   private static byte [] VALUE = Bytes.toBytes("testValue");
 
   @Rule
-  public TestName name = new TestName();
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    conf.setLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, 10 * 1024 * 1024);
-    TEST_UTIL.startMiniCluster(3);
-  }
+  public TestTableName name = new TestTableName();
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
+    if (TEST_UTIL != null) {
+      TEST_UTIL.shutdownMiniCluster();
+    }
   }
 
-  @Before
-  public void setUp() throws Exception {
-    // Nothing to do.
+  @Parameterized.Parameters
+  public static Collection parameters() {
+    return Arrays.asList(new Object[][] {
+        { MasterRegistry.class, 1},
+        { MasterRegistry.class, 2},
+        { ZKConnectionRegistry.class, 1}
+    });
   }
 
   /**
-   * @throws java.lang.Exception
+   * JUnit does not provide an easy way to run a hook after each parameterized run. Without that
+   * there is no easy way to restart the test cluster after each parameterized run. Annotation
+   * BeforeParam does not work either because it runs before parameterization and hence does not
+   * have access to the test parameters (which is weird).
+   *
+   * This *hack* checks if the current instance of test cluster configuration has the passed
+   * parameterized configs. In such a case, we can just reuse the cluster for test and do not need
+   * to initialize from scratch. While this is a hack, it saves a ton of time for the full
+   * test and de-flakes it.
    */
-  @After
-  public void tearDown() throws Exception {
-    // Nothing to do.
+  private static boolean isSameParameterizedCluster(Class registryImpl, int numHedgedReqs) {
+    // initialize() is called for every unit test, however we only want to reset the cluster state
+    // at the end of every parameterized run.
+    if (TEST_UTIL == null) {
+      return false;
+    }
+    Configuration conf = TEST_UTIL.getConfiguration();
+    Class confClass = conf.getClass(
+        HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class);
+    int hedgedReqConfig = conf.getInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY,
+        HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT);
+    return confClass.getName().equals(registryImpl.getName()) && numHedgedReqs == hedgedReqConfig;
+  }
+
+  public TestScannersFromClientSide(Class registryImpl, int numHedgedReqs) throws Exception {
+    if (isSameParameterizedCluster(registryImpl, numHedgedReqs)) {
+      return;
+    }
+    if (TEST_UTIL != null) {
+      // We reached the end of a parameterized run, clean up the cluster.
+      TEST_UTIL.shutdownMiniCluster();
+    }
+    TEST_UTIL = new HBaseTestingUtility();
+    Configuration conf = TEST_UTIL.getConfiguration();
+    conf.setLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, 10 * 1024 * 1024);
+    conf.setClass(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, registryImpl,
+        ConnectionRegistry.class);
+    if (numHedgedReqs == 1) {
+      conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, false);
+    } else {
+      Preconditions.checkArgument(numHedgedReqs > 1);
+      conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, true);
+    }
+    conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, numHedgedReqs);
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    // Multiple masters needed only when hedged reads for master registry are enabled.
+    builder.numMasters(numHedgedReqs > 1 ? 3 : 1).numRegionServers(3);
+    TEST_UTIL.startMiniCluster(builder.build());
   }
 
   /**
@@ -120,7 +166,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testScanBatch() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 8);
 
     Table ht = TEST_UTIL.createTable(tableName, FAMILY);
@@ -190,7 +236,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testMaxResultSizeIsSetToDefault() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     Table ht = TEST_UTIL.createTable(tableName, FAMILY);
 
     // The max result size we expect the scan to use by default.
@@ -259,7 +305,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testSmallScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     int numRows = 10;
     byte[][] ROWS = HTestConst.makeNAscii(ROW, numRows);
@@ -292,7 +338,8 @@ public class TestScannersFromClientSide {
   /**
    * Run through a variety of test configurations with a small scan
    */
-  private void testSmallScan(Table table, boolean reversed, int rows, int columns) throws Exception {
+  private void testSmallScan(
+      Table table, boolean reversed, int rows, int columns) throws Exception {
     Scan baseScan = new Scan();
     baseScan.setReversed(reversed);
     baseScan.setSmall(true);
@@ -334,7 +381,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testGetMaxResults() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20);
 
@@ -408,8 +455,8 @@ public class TestScannersFromClientSide {
       kvListExp.add(new KeyValue(ROW, FAMILIES[1], QUALIFIERS[i], 1, VALUE));
     }
     for (int i=0; i < 2; i++) {
-        kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE));
-      }
+      kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE));
+    }
     for (int i=10; i < 20; i++) {
       kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE));
     }
@@ -452,7 +499,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testScanMaxResults() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] ROWS = HTestConst.makeNAscii(ROW, 2);
     byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 10);
@@ -500,7 +547,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testGetRowOffset() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20);
 
@@ -519,7 +566,9 @@ public class TestScannersFromClientSide {
       KeyValue kv = new KeyValue(ROW, FAMILIES[0], QUALIFIERS[i], 1, VALUE);
       put.add(kv);
       // skipping first two kvs
-      if (i < 2) continue;
+      if (i < 2) {
+        continue;
+      }
       kvListExp.add(kv);
     }
     ht.put(put);
@@ -590,7 +639,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testScanRawDeleteFamilyVersion() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     TEST_UTIL.createTable(tableName, FAMILY);
     Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
     conf.set(RPC_CODEC_CONF_KEY, "");
@@ -618,7 +667,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testScanOnReopenedRegion() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 2);
 
     Table ht = TEST_UTIL.createTable(tableName, FAMILY);
@@ -693,8 +742,9 @@ public class TestScannersFromClientSide {
     LOG.info(msg);
     LOG.info("Expected count: " + expKvList.size());
     LOG.info("Actual count: " + result.size());
-    if (expKvList.isEmpty())
+    if (expKvList.isEmpty()) {
       return;
+    }
 
     int i = 0;
     for (Cell kv : result.rawCells()) {
@@ -715,7 +765,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testReadExpiredDataForRawScan() throws IOException {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     long ts = System.currentTimeMillis() - 10000;
     byte[] value = Bytes.toBytes("expired");
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
@@ -735,7 +785,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testScanWithColumnsAndFilterAndVersion() throws IOException {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY, 4)) {
       for (int i = 0; i < 4; i++) {
         Put put = new Put(ROW);
@@ -757,7 +807,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testScanWithSameStartRowStopRow() throws IOException {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
       table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE));
 
@@ -794,7 +844,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testReverseScanWithFlush() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     final int BATCH_SIZE = 10;
     final int ROWS_TO_INSERT = 100;
     final byte[] LARGE_VALUE = generateHugeValue(128 * 1024);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index ac0d356..2797df3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -30,22 +30,31 @@ import static org.mockito.Matchers.anyObject;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 import static org.mockito.internal.verification.VerificationModeFactory.times;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.util.StringUtils;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+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.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto;
@@ -54,14 +63,6 @@ import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseReq
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.Interface;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.util.StringUtils;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
  * Some basic ipc tests.
@@ -232,7 +233,6 @@ public abstract class AbstractTestIPC {
   /**
    * Tests that the RpcServer creates & dispatches CallRunner object to scheduler with non-null
    * remoteAddress set to its Call Object
-   * @throws ServiceException
    */
   @Test
   public void testRpcServerForNotNullRemoteAddressInCallObject()
@@ -363,6 +363,104 @@ public abstract class AbstractTestIPC {
     }
   }
 
+  /**
+   * Tests the various request fan out values using a simple RPC hedged across a mix of running and
+   * failing servers.
+   */
+  @Test
+  public void testHedgedAsyncEcho() throws Exception {
+    // Hedging is not supported for blocking connection types.
+    Assume.assumeFalse(this instanceof TestBlockingIPC);
+    List<RpcServer> rpcServers = new ArrayList<>();
+    List<InetSocketAddress> addresses = new ArrayList<>();
+    // Create a mix of running and failing servers.
+    final int numRunningServers = 5;
+    final int numFailingServers = 3;
+    final int numServers = numRunningServers + numFailingServers;
+    for (int i = 0; i < numRunningServers; i++) {
+      RpcServer rpcServer = createRpcServer(null, "testRpcServer" + i,
+          Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
+          SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
+          new FifoRpcScheduler(CONF, 1));
+      rpcServer.start();
+      addresses.add(rpcServer.getListenerAddress());
+      rpcServers.add(rpcServer);
+    }
+    for (int i = 0; i < numFailingServers; i++) {
+      RpcServer rpcServer = createTestFailingRpcServer(null, "testFailingRpcServer" + i,
+          Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
+          SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
+          new FifoRpcScheduler(CONF, 1));
+      rpcServer.start();
+      addresses.add(rpcServer.getListenerAddress());
+      rpcServers.add(rpcServer);
+    }
+    Configuration conf = HBaseConfiguration.create();
+    try (AbstractRpcClient<?> client = createRpcClient(conf)) {
+      // Try out various fan out values starting from 1 -> numServers.
+      for (int reqFanOut = 1; reqFanOut <= numServers; reqFanOut++) {
+        // Update the client's underlying conf, should be ok for the test.
+        LOG.debug("Testing with request fan out: " + reqFanOut);
+        conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, reqFanOut);
+        Interface stub = newStub(client, addresses);
+        BlockingRpcCallback<EchoResponseProto> done = new BlockingRpcCallback<>();
+        stub.echo(new HBaseRpcControllerImpl(),
+            EchoRequestProto.newBuilder().setMessage("hello").build(), done);
+        TestProtos.EchoResponseProto responseProto = done.get();
+        assertNotNull(responseProto);
+        assertEquals("hello", responseProto.getMessage());
+        LOG.debug("Ended test with request fan out: " + reqFanOut);
+      }
+    } finally {
+      for (RpcServer rpcServer: rpcServers) {
+        rpcServer.stop();
+      }
+    }
+  }
+
+  @Test
+  public void testHedgedAsyncTimeouts() throws Exception {
+    // Hedging is not supported for blocking connection types.
+    Assume.assumeFalse(this instanceof TestBlockingIPC);
+    List<RpcServer> rpcServers = new ArrayList<>();
+    List<InetSocketAddress> addresses = new ArrayList<>();
+    final int numServers = 3;
+    for (int i = 0; i < numServers; i++) {
+      RpcServer rpcServer = createRpcServer(null, "testTimeoutRpcServer" + i,
+          Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
+          SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
+          new FifoRpcScheduler(CONF, 1));
+      rpcServer.start();
+      addresses.add(rpcServer.getListenerAddress());
+      rpcServers.add(rpcServer);
+    }
+    Configuration conf = HBaseConfiguration.create();
+    int timeout = 100;
+    int pauseTime = 1000;
+    try (AbstractRpcClient<?> client = createRpcClient(conf)) {
+      // Try out various fan out values starting from 1 -> numServers.
+      for (int reqFanOut = 1; reqFanOut <= numServers; reqFanOut++) {
+        // Update the client's underlying conf, should be ok for the test.
+        LOG.debug("Testing with request fan out: " + reqFanOut);
+        conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, reqFanOut);
+        Interface stub = newStub(client, addresses);
+        HBaseRpcController pcrc = new HBaseRpcControllerImpl();
+        pcrc.setCallTimeout(timeout);
+        BlockingRpcCallback<EmptyResponseProto> callback = new BlockingRpcCallback<>();
+        stub.pause(pcrc, PauseRequestProto.newBuilder().setMs(pauseTime).build(), callback);
+        assertNull(callback.get());
+        // Make sure the controller has the right exception propagated.
+        assertTrue(pcrc.getFailed() instanceof CallTimeoutException);
+        LOG.debug("Ended test with request fan out: " + reqFanOut);
+      }
+    } finally {
+      for (RpcServer rpcServer: rpcServers) {
+        rpcServer.stop();
+      }
+    }
+  }
+
+
   @Test
   public void testAsyncRemoteError() throws IOException {
     AbstractRpcClient<?> client = createRpcClient(CONF);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java
index d8a2d34..6adfa46 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java
@@ -17,21 +17,23 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
-
+import java.util.Set;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto;
@@ -41,8 +43,6 @@ import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseReq
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.Interface;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.Threads;
 
 @InterfaceAudience.Private
 public class TestProtobufRpcServiceImpl implements BlockingInterface {
@@ -67,6 +67,17 @@ public class TestProtobufRpcServiceImpl implements BlockingInterface {
       User.getCurrent(), 0));
   }
 
+  public static Interface newStub(RpcClient client, List<InetSocketAddress> addrs)
+      throws IOException {
+    Set<ServerName> serverNames = new HashSet<>();
+    for (InetSocketAddress addr: addrs) {
+      serverNames.add(ServerName.valueOf(
+          addr.getHostName(), addr.getPort(), System.currentTimeMillis()));
+    }
+    return TestProtobufRpcProto.newStub(client.createHedgedRpcChannel(
+        serverNames, User.getCurrent(), 0));
+  }
+
   @Override
   public EmptyResponseProto ping(RpcController controller, EmptyRequestProto request)
       throws ServiceException {


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

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 8d4314fd2cf55e22dcaebaf13c401749e5a2dbd1
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   |   3 +-
 .../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   |  38 +---
 .../apache/hadoop/hbase/zookeeper/ZKWatcher.java   |  37 ++-
 23 files changed, 586 insertions(+), 86 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 c8511d4..d923ebf 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
@@ -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
@@ -40,7 +40,8 @@ public class ZNodePaths {
   // TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
   public static final char ZNODE_PATH_SEPARATOR = '/';
 
-  public final static 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
@@ -94,7 +95,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);
@@ -179,6 +180,18 @@ public class ZNodePaths {
   }
 
   /**
+   * 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 3570140..bffb077 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
@@ -360,6 +360,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;
 
@@ -513,8 +519,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 {
@@ -527,7 +532,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(),
@@ -575,8 +579,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);
@@ -3887,4 +3893,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 a6844fc..84722df 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
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.zookeeper.KeeperException;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -147,7 +146,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 10fd165..be2e5c4 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;
 
 /**
@@ -274,42 +269,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;


[hbase] 05/06: HBASE-23648: Re-use underlying connection registry in RawAsyncHBaseAdmin (#994)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 49fb4518eb87d3b7ab57d8f92f14a0da1ea8cbd3
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Thu Jan 9 12:27:09 2020 -0800

    HBASE-23648: Re-use underlying connection registry in RawAsyncHBaseAdmin (#994)
    
    * HBASE-23648: Re-use underlying connection registry in RawAsyncHBaseAdmin
    
    No need to create and close a new registry on demand. Audited other
    usages of getRegistry() and the code looks fine.
    
    * Fix checkstyle issues in RawAsyncHBaseAdmin
---
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    | 110 +++++++++------------
 1 file changed, 47 insertions(+), 63 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 75324c9..fb9d150 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -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
@@ -21,7 +21,6 @@ import static org.apache.hadoop.hbase.HConstants.HIGH_QOS;
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 import static org.apache.hadoop.hbase.util.FutureUtils.unwrapCompletionException;
-
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcChannel;
 import java.io.IOException;
@@ -46,7 +45,6 @@ import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
 import org.apache.hadoop.hbase.CacheEvictionStats;
@@ -99,14 +97,12 @@ import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.yetus.audience.InterfaceAudience;
 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.Preconditions;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
 import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
 import org.apache.hbase.thirdparty.io.netty.util.Timeout;
 import org.apache.hbase.thirdparty.io.netty.util.TimerTask;
-
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
@@ -749,7 +745,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) {
+  public CompletableFuture<Void> addColumnFamily(
+      TableName tableName, ColumnFamilyDescriptor columnFamily) {
     return this.<AddColumnRequest, AddColumnResponse> procedureCall(tableName,
       RequestConverter.buildAddColumnRequest(tableName, columnFamily, ng.getNonceGroup(),
         ng.newNonce()), (s, c, req, done) -> s.addColumn(c, req, done), (resp) -> resp.getProcId(),
@@ -803,10 +800,10 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         .<NamespaceDescriptor> newMasterCaller()
         .action(
           (controller, stub) -> this
-              .<GetNamespaceDescriptorRequest, GetNamespaceDescriptorResponse, NamespaceDescriptor> call(
-                controller, stub, RequestConverter.buildGetNamespaceDescriptorRequest(name), (s, c,
-                    req, done) -> s.getNamespaceDescriptor(c, req, done), (resp) -> ProtobufUtil
-                    .toNamespaceDescriptor(resp.getNamespaceDescriptor()))).call();
+              .<GetNamespaceDescriptorRequest, GetNamespaceDescriptorResponse, NamespaceDescriptor>
+                  call(controller, stub, RequestConverter.buildGetNamespaceDescriptorRequest(name),
+                    (s, c, req, done) -> s.getNamespaceDescriptor(c, req, done), (resp)
+                      -> ProtobufUtil.toNamespaceDescriptor(resp.getNamespaceDescriptor()))).call();
   }
 
   @Override
@@ -824,13 +821,12 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors() {
     return this
-        .<List<NamespaceDescriptor>> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<ListNamespaceDescriptorsRequest, ListNamespaceDescriptorsResponse, List<NamespaceDescriptor>> call(
-                controller, stub, ListNamespaceDescriptorsRequest.newBuilder().build(), (s, c, req,
-                    done) -> s.listNamespaceDescriptors(c, req, done), (resp) -> ProtobufUtil
-                    .toNamespaceDescriptorList(resp))).call();
+        .<List<NamespaceDescriptor>> newMasterCaller().action((controller, stub) -> this
+              .<ListNamespaceDescriptorsRequest, ListNamespaceDescriptorsResponse,
+                  List<NamespaceDescriptor>> call(controller, stub,
+                  ListNamespaceDescriptorsRequest.newBuilder().build(), (s, c, req, done) ->
+                      s.listNamespaceDescriptors(c, req, done),
+                    (resp) -> ProtobufUtil.toNamespaceDescriptorList(resp))).call();
   }
 
   @Override
@@ -1074,10 +1070,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   private CompletableFuture<List<HRegionLocation>> getTableHRegionLocations(TableName tableName) {
     if (TableName.META_TABLE_NAME.equals(tableName)) {
       CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
-      // For meta table, we use zk to fetch all locations.
-      ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(
-          connection.getConfiguration());
-      addListener(registry.getMetaRegionLocations(), (metaRegions, err) -> {
+      addListener(connection.registry.getMetaRegionLocations(), (metaRegions, err) -> {
         if (err != null) {
           future.completeExceptionally(err);
         } else if (metaRegions == null || metaRegions.isEmpty() ||
@@ -1086,8 +1079,6 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         } else {
           future.complete(Collections.singletonList(metaRegions.getDefaultRegionLocation()));
         }
-        // close the registry.
-        IOUtils.closeQuietly(registry);
       });
       return future;
     } else {
@@ -1683,11 +1674,10 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String peerId) {
     return this.<ReplicationPeerConfig> newMasterCaller().action((controller, stub) -> this
-      .<GetReplicationPeerConfigRequest, GetReplicationPeerConfigResponse, ReplicationPeerConfig> call(
-        controller, stub, RequestConverter.buildGetReplicationPeerConfigRequest(peerId),
-        (s, c, req, done) -> s.getReplicationPeerConfig(c, req, done),
-        (resp) -> ReplicationPeerConfigUtil.convert(resp.getPeerConfig())))
-      .call();
+      .<GetReplicationPeerConfigRequest, GetReplicationPeerConfigResponse, ReplicationPeerConfig>
+          call(controller, stub, RequestConverter.buildGetReplicationPeerConfigRequest(peerId),
+            (s, c, req, done) -> s.getReplicationPeerConfig(c, req, done),
+            (resp) -> ReplicationPeerConfigUtil.convert(resp.getPeerConfig()))).call();
   }
 
   @Override
@@ -1704,13 +1694,13 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId,
       SyncReplicationState clusterState) {
-    return this
-      .<TransitReplicationPeerSyncReplicationStateRequest, TransitReplicationPeerSyncReplicationStateResponse> procedureCall(
+    return this.<TransitReplicationPeerSyncReplicationStateRequest,
+        TransitReplicationPeerSyncReplicationStateResponse> procedureCall(
         RequestConverter.buildTransitReplicationPeerSyncReplicationStateRequest(peerId,
           clusterState),
-        (s, c, req, done) -> s.transitReplicationPeerSyncReplicationState(c, req, done),
-        (resp) -> resp.getProcId(), new ReplicationProcedureBiConsumer(peerId,
-          () -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE"));
+          (s, c, req, done) -> s.transitReplicationPeerSyncReplicationState(c, req, done),
+          (resp) -> resp.getProcId(), new ReplicationProcedureBiConsumer(peerId,
+            () -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE"));
   }
 
   @Override
@@ -1780,11 +1770,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     return this
         .<List<ReplicationPeerDescription>> newMasterCaller()
         .action(
-          (controller, stub) -> this
-              .<ListReplicationPeersRequest, ListReplicationPeersResponse, List<ReplicationPeerDescription>> call(
-                controller,
-                stub,
-                request,
+          (controller, stub) -> this.<ListReplicationPeersRequest, ListReplicationPeersResponse,
+              List<ReplicationPeerDescription>> call(controller, stub, request,
                 (s, c, req, done) -> s.listReplicationPeers(c, req, done),
                 (resp) -> resp.getPeerDescList().stream()
                     .map(ReplicationPeerConfigUtil::toReplicationPeerDescription)
@@ -2293,11 +2280,13 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> decommissionRegionServers(List<ServerName> servers, boolean offload) {
+  public CompletableFuture<Void> decommissionRegionServers(
+      List<ServerName> servers, boolean offload) {
     return this.<Void> newMasterCaller()
         .action((controller, stub) -> this
           .<DecommissionRegionServersRequest, DecommissionRegionServersResponse, Void> call(
-            controller, stub, RequestConverter.buildDecommissionRegionServersRequest(servers, offload),
+            controller, stub,
+              RequestConverter.buildDecommissionRegionServersRequest(servers, offload),
             (s, c, req, done) -> s.decommissionRegionServers(c, req, done), resp -> null))
         .call();
   }
@@ -2319,11 +2308,11 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   public CompletableFuture<Void> recommissionRegionServer(ServerName server,
       List<byte[]> encodedRegionNames) {
     return this.<Void> newMasterCaller()
-        .action((controller, stub) -> this
-          .<RecommissionRegionServerRequest, RecommissionRegionServerResponse, Void> call(controller,
-            stub, RequestConverter.buildRecommissionRegionServerRequest(server, encodedRegionNames),
-            (s, c, req, done) -> s.recommissionRegionServer(c, req, done), resp -> null))
-        .call();
+        .action((controller, stub) ->
+            this.<RecommissionRegionServerRequest, RecommissionRegionServerResponse, Void> call(
+                controller, stub, RequestConverter.buildRecommissionRegionServerRequest(
+                    server, encodedRegionNames), (s, c, req, done) -> s.recommissionRegionServer(
+                        c, req, done), resp -> null)).call();
   }
 
   /**
@@ -2389,7 +2378,6 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
    * Get the region info for the passed region name. The region name may be a full region name or
    * encoded region name. If the region does not found, then it'll throw an UnknownRegionException
    * wrapped by a {@link CompletableFuture}
-   * @param regionNameOrEncodedRegionName
    * @return region info, wrapped by a {@link CompletableFuture}
    */
   private CompletableFuture<RegionInfo> getRegionInfo(byte[] regionNameOrEncodedRegionName) {
@@ -2880,10 +2868,11 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         .<List<SecurityCapability>> newMasterCaller()
         .action(
           (controller, stub) -> this
-              .<SecurityCapabilitiesRequest, SecurityCapabilitiesResponse, List<SecurityCapability>> call(
-                controller, stub, SecurityCapabilitiesRequest.newBuilder().build(), (s, c, req,
-                    done) -> s.getSecurityCapabilities(c, req, done), (resp) -> ProtobufUtil
-                    .toSecurityCapabilityList(resp.getCapabilitiesList()))).call();
+              .<SecurityCapabilitiesRequest, SecurityCapabilitiesResponse, List<SecurityCapability>>
+                  call(controller, stub, SecurityCapabilitiesRequest.newBuilder().build(),
+                    (s, c, req, done) -> s.getSecurityCapabilities(c, req, done),
+                    (resp) -> ProtobufUtil.toSecurityCapabilityList(resp.getCapabilitiesList())))
+        .call();
   }
 
   @Override
@@ -3060,14 +3049,10 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     MajorCompactionTimestampRequest request =
         MajorCompactionTimestampRequest.newBuilder()
             .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
-    return this
-        .<Optional<Long>> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<MajorCompactionTimestampRequest, MajorCompactionTimestampResponse, Optional<Long>> call(
-                controller, stub, request,
-                (s, c, req, done) -> s.getLastMajorCompactionTimestamp(c, req, done),
-                ProtobufUtil::toOptionalTimestamp)).call();
+    return this.<Optional<Long>> newMasterCaller().action((controller, stub) ->
+        this.<MajorCompactionTimestampRequest, MajorCompactionTimestampResponse, Optional<Long>>
+            call(controller, stub, request, (s, c, req, done) -> s.getLastMajorCompactionTimestamp(
+                c, req, done), ProtobufUtil::toOptionalTimestamp)).call();
   }
 
   @Override
@@ -3207,11 +3192,10 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   public CompletableFuture<Boolean> isBalancerEnabled() {
     return this
         .<Boolean> newMasterCaller()
-        .action(
-          (controller, stub) -> this.<IsBalancerEnabledRequest, IsBalancerEnabledResponse, Boolean> call(
-            controller, stub, RequestConverter.buildIsBalancerEnabledRequest(),
-            (s, c, req, done) -> s.isBalancerEnabled(c, req, done), (resp) -> resp.getEnabled()))
-        .call();
+        .action((controller, stub) ->
+              this.<IsBalancerEnabledRequest, IsBalancerEnabledResponse, Boolean> call(controller,
+            stub, RequestConverter.buildIsBalancerEnabledRequest(), (s, c, req, done)
+                  -> s.isBalancerEnabled(c, req, done), (resp) -> resp.getEnabled())).call();
   }
 
   @Override


[hbase] 01/06: HBASE-23275: Track active master's address in ActiveMasterManager (#812)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 9c42b6a17b18086068dc0aa149fe54bc0ed4a212
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Wed Nov 20 11:41:36 2019 -0800

    HBASE-23275: Track active master's address in ActiveMasterManager (#812)
    
    * HBASE-23275: Track active master's address in ActiveMasterManager
    
    Currently we just track whether an active master exists.
    It helps to also track the address of the active master in
    all the masters to help serve the client RPC requests to
    know which master is active.
    
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../hadoop/hbase/master/ActiveMasterManager.java   | 63 +++++++++++++++++-----
 .../org/apache/hadoop/hbase/master/HMaster.java    |  4 ++
 .../hbase/master/TestActiveMasterManager.java      | 10 ++++
 3 files changed, 64 insertions(+), 13 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
index 50798ed..99cab62 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -17,25 +17,24 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hbase.master;
-
 import java.io.IOException;
+import java.util.Optional;
 import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-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.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZNodeClearer;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 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.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 
 /**
  * Handles everything on master-side related to master election.
@@ -57,12 +56,18 @@ public class ActiveMasterManager extends ZKListener {
   final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
   final AtomicBoolean clusterShutDown = new AtomicBoolean(false);
 
+  // This server's information.
   private final ServerName sn;
   private int infoPort;
   private final Server master;
 
+  // Active master's server name. Invalidated anytime active master changes (based on ZK
+  // notifications) and lazily fetched on-demand.
+  // ServerName is immutable, so we don't need heavy synchronization around it.
+  private volatile ServerName activeMasterServerName;
+
   /**
-   * @param watcher
+   * @param watcher ZK watcher
    * @param sn ServerName
    * @param master In an instance of a Master.
    */
@@ -107,6 +112,30 @@ public class ActiveMasterManager extends ZKListener {
   }
 
   /**
+   * Fetches the active master's ServerName from zookeeper.
+   */
+  private void fetchAndSetActiveMasterServerName() {
+    LOG.debug("Attempting to fetch active master sn from zk");
+    try {
+      activeMasterServerName = MasterAddressTracker.getMasterAddress(watcher);
+    } catch (IOException | KeeperException e) {
+      // Log and ignore for now and re-fetch later if needed.
+      LOG.error("Error fetching active master information", e);
+    }
+  }
+
+  public Optional<ServerName> getActiveMasterServerName() {
+    if (!clusterHasActiveMaster.get()) {
+      return Optional.empty();
+    }
+    if (activeMasterServerName == null) {
+      fetchAndSetActiveMasterServerName();
+    }
+    // It could still be null, but return whatever we have.
+    return Optional.ofNullable(activeMasterServerName);
+  }
+
+  /**
    * Handle a change in the master node.  Doesn't matter whether this was called
    * from a nodeCreated or nodeDeleted event because there are no guarantees
    * that the current state of the master node matches the event at the time of
@@ -134,6 +163,9 @@ public class ActiveMasterManager extends ZKListener {
           // Notify any thread waiting to become the active master
           clusterHasActiveMaster.notifyAll();
         }
+        // Reset the active master sn. Will be re-fetched later if needed.
+        // We don't want to make a synchronous RPC under a monitor.
+        activeMasterServerName = null;
       }
     } catch (KeeperException ke) {
       master.abort("Received an unexpected KeeperException, aborting", ke);
@@ -151,8 +183,8 @@ public class ActiveMasterManager extends ZKListener {
    * @param checkInterval the interval to check if the master is stopped
    * @param startupStatus the monitor status to track the progress
    * @return True if no issue becoming active master else false if another
-   * master was running or if some other problem (zookeeper, stop flag has been
-   * set on this Master)
+   *   master was running or if some other problem (zookeeper, stop flag has been
+   *   set on this Master)
    */
   boolean blockUntilBecomingActiveMaster(
       int checkInterval, MonitoredTask startupStatus) {
@@ -178,10 +210,14 @@ public class ActiveMasterManager extends ZKListener {
           // We are the master, return
           startupStatus.setStatus("Successfully registered as active master.");
           this.clusterHasActiveMaster.set(true);
+          activeMasterServerName = sn;
           LOG.info("Registered as active master=" + this.sn);
           return true;
         }
 
+        // Invalidate the active master name so that subsequent requests do not get any stale
+        // master information. Will be re-fetched if needed.
+        activeMasterServerName = null;
         // There is another active master running elsewhere or this is a restart
         // and the master ephemeral node has not expired yet.
         this.clusterHasActiveMaster.set(true);
@@ -208,7 +244,8 @@ public class ActiveMasterManager extends ZKListener {
             ZKUtil.deleteNode(this.watcher, this.watcher.getZNodePaths().masterAddressZNode);
 
             // We may have failed to delete the znode at the previous step, but
-            //  we delete the file anyway: a second attempt to delete the znode is likely to fail again.
+            //  we delete the file anyway: a second attempt to delete the znode is likely to fail
+            //  again.
             ZNodeClearer.deleteMyEphemeralNodeOnDisk();
           } else {
             msg = "Another master is the active master, " + currentMaster +
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 b5f19d8..3570140 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
@@ -3876,6 +3876,10 @@ public class HMaster extends HRegionServer implements MasterServices {
     return cachedClusterId.getFromCacheOrFetch();
   }
 
+  public Optional<ServerName> getActiveMaster() {
+    return activeMasterManager.getActiveMasterServerName();
+  }
+
   @Override
   public void runReplicationBarrierCleaner() {
     ReplicationBarrierCleaner rbc = this.replicationBarrierCleaner;
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 4649eea..84837f9 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -91,6 +92,7 @@ public class TestActiveMasterManager {
     ActiveMasterManager activeMasterManager =
       dummyMaster.getActiveMasterManager();
     assertFalse(activeMasterManager.clusterHasActiveMaster.get());
+    assertFalse(activeMasterManager.getActiveMasterServerName().isPresent());
 
     // First test becoming the active master uninterrupted
     MonitoredTask status = Mockito.mock(MonitoredTask.class);
@@ -99,6 +101,7 @@ public class TestActiveMasterManager {
     activeMasterManager.blockUntilBecomingActiveMaster(100, status);
     assertTrue(activeMasterManager.clusterHasActiveMaster.get());
     assertMaster(zk, master);
+    assertMaster(zk, activeMasterManager.getActiveMasterServerName().get());
 
     // Now pretend master restart
     DummyMaster secondDummyMaster = new DummyMaster(zk,master);
@@ -108,6 +111,8 @@ public class TestActiveMasterManager {
     activeMasterManager.blockUntilBecomingActiveMaster(100, status);
     assertTrue(activeMasterManager.clusterHasActiveMaster.get());
     assertMaster(zk, master);
+    assertMaster(zk, activeMasterManager.getActiveMasterServerName().get());
+    assertMaster(zk, secondActiveMasterManager.getActiveMasterServerName().get());
   }
 
   /**
@@ -135,6 +140,7 @@ public class TestActiveMasterManager {
     ActiveMasterManager activeMasterManager =
       ms1.getActiveMasterManager();
     assertFalse(activeMasterManager.clusterHasActiveMaster.get());
+    assertFalse(activeMasterManager.getActiveMasterServerName().isPresent());
 
     // First test becoming the active master uninterrupted
     ClusterStatusTracker clusterStatusTracker =
@@ -144,6 +150,7 @@ public class TestActiveMasterManager {
         Mockito.mock(MonitoredTask.class));
     assertTrue(activeMasterManager.clusterHasActiveMaster.get());
     assertMaster(zk, firstMasterAddress);
+    assertMaster(zk, activeMasterManager.getActiveMasterServerName().get());
 
     // New manager will now try to become the active master in another thread
     WaitToBeMasterThread t = new WaitToBeMasterThread(zk, secondMasterAddress);
@@ -161,6 +168,8 @@ public class TestActiveMasterManager {
     assertTrue(t.manager.clusterHasActiveMaster.get());
     // But secondary one should not be the active master
     assertFalse(t.isActiveMaster);
+    // Verify the active master ServerName is populated in standby master.
+    assertEquals(firstMasterAddress, t.manager.getActiveMasterServerName().get());
 
     // Close the first server and delete it's master node
     ms1.stop("stopping first server");
@@ -189,6 +198,7 @@ public class TestActiveMasterManager {
 
     assertTrue(t.manager.clusterHasActiveMaster.get());
     assertTrue(t.isActiveMaster);
+    assertEquals(secondMasterAddress, t.manager.getActiveMasterServerName().get());
 
     LOG.info("Deleting master node");
 


[hbase] 03/06: HBASE-23304: RPCs needed for client meta information lookup (#904)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit bc891a868cdeabb598b591138f489b1244199362
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Thu Dec 19 11:29:25 2019 -0800

    HBASE-23304: RPCs needed for client meta information lookup (#904)
    
    * HBASE-23304: RPCs needed for client meta information lookup
    
    This patch implements the RPCs needed for the meta information
    lookup during connection init. New tests added to cover the RPC
    code paths. HBASE-23305 builds on this to implement the client
    side logic.
    
    Fixed a bunch of checkstyle nits around the places the patch
    touches.
    
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java |   4 +-
 .../src/main/protobuf/Master.proto                 |  44 ++++++
 .../hadoop/hbase/master/MasterRpcServices.java     |  85 ++++++++---
 .../hbase/master/TestClientMetaServiceRPCs.java    | 164 +++++++++++++++++++++
 4 files changed, 275 insertions(+), 22 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 2adcea9..23f5c08 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
@@ -376,7 +376,9 @@ public final class ProtobufUtil {
    * @see #toServerName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName)
    */
   public static HBaseProtos.ServerName toServerName(final ServerName serverName) {
-    if (serverName == null) return null;
+    if (serverName == null) {
+      return null;
+    }
     HBaseProtos.ServerName.Builder builder =
       HBaseProtos.ServerName.newBuilder();
     builder.setHostName(serverName.getHostname());
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 69377a6..e88ddc4 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -1200,3 +1200,47 @@ service HbckService {
   rpc FixMeta(FixMetaRequest)
     returns(FixMetaResponse);
 }
+
+/** Request and response to get the clusterID for this cluster */
+message GetClusterIdRequest {
+}
+message GetClusterIdResponse {
+  /** Not set if cluster ID could not be determined. */
+  optional string cluster_id = 1;
+}
+
+/** Request and response to get the currently active master name for this cluster */
+message GetActiveMasterRequest {
+}
+message GetActiveMasterResponse {
+  /** Not set if an active master could not be determined. */
+  optional ServerName server_name = 1;
+}
+
+/** Request and response to get the current list of meta region locations */
+message GetMetaRegionLocationsRequest {
+}
+message GetMetaRegionLocationsResponse {
+  /** Not set if meta region locations could not be determined. */
+  repeated RegionLocation meta_locations = 1;
+}
+
+/**
+ * Implements all the RPCs needed by clients to look up cluster meta information needed for connection establishment.
+ */
+service ClientMetaService {
+  /**
+   * Get Cluster ID for this cluster.
+   */
+  rpc GetClusterId(GetClusterIdRequest) returns(GetClusterIdResponse);
+
+  /**
+   * Get active master server name for this cluster.
+   */
+  rpc GetActiveMaster(GetActiveMasterRequest) returns(GetActiveMasterResponse);
+
+  /**
+   * Get current meta replicas' region locations.
+   */
+  rpc GetMetaRegionLocations(GetMetaRegionLocationsRequest) returns(GetMetaRegionLocationsResponse);
+}
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 612c731..620c3a0 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
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.master;
 
 import static org.apache.hadoop.hbase.master.MasterWalManager.META_FILTER;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.BindException;
@@ -30,6 +29,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
 import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.Server;
@@ -116,11 +117,9 @@ 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.RpcController;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
-
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
@@ -161,6 +160,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceReq
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClientMetaService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
@@ -185,12 +185,18 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProced
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
@@ -351,9 +357,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.Snapshot
  */
 @InterfaceAudience.Private
 @SuppressWarnings("deprecation")
-public class MasterRpcServices extends RSRpcServices
-      implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface,
-        LockService.BlockingInterface, HbckService.BlockingInterface {
+public class MasterRpcServices extends RSRpcServices implements
+    MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface,
+    LockService.BlockingInterface, HbckService.BlockingInterface,
+    ClientMetaService.BlockingInterface {
   private static final Logger LOG = LoggerFactory.getLogger(MasterRpcServices.class.getName());
   private static final Logger AUDITLOG =
       LoggerFactory.getLogger("SecurityLogger."+MasterRpcServices.class.getName());
@@ -362,7 +369,7 @@ public class MasterRpcServices extends RSRpcServices
 
   /**
    * @return Subset of configuration to pass initializing regionservers: e.g.
-   * the filesystem to use and root directory to use.
+   *     the filesystem to use and root directory to use.
    */
   private RegionServerStartupResponse.Builder createConfigurationSubset() {
     RegionServerStartupResponse.Builder resp = addConfig(
@@ -488,15 +495,17 @@ public class MasterRpcServices extends RSRpcServices
   protected List<BlockingServiceAndInterface> getServices() {
     List<BlockingServiceAndInterface> bssi = new ArrayList<>(5);
     bssi.add(new BlockingServiceAndInterface(
-      MasterService.newReflectiveBlockingService(this),
-      MasterService.BlockingInterface.class));
+        MasterService.newReflectiveBlockingService(this),
+        MasterService.BlockingInterface.class));
     bssi.add(new BlockingServiceAndInterface(
-      RegionServerStatusService.newReflectiveBlockingService(this),
-      RegionServerStatusService.BlockingInterface.class));
+        RegionServerStatusService.newReflectiveBlockingService(this),
+        RegionServerStatusService.BlockingInterface.class));
     bssi.add(new BlockingServiceAndInterface(LockService.newReflectiveBlockingService(this),
         LockService.BlockingInterface.class));
     bssi.add(new BlockingServiceAndInterface(HbckService.newReflectiveBlockingService(this),
         HbckService.BlockingInterface.class));
+    bssi.add(new BlockingServiceAndInterface(ClientMetaService.newReflectiveBlockingService(this),
+        ClientMetaService.BlockingInterface.class));
     bssi.addAll(super.getServices());
     return bssi;
   }
@@ -623,7 +632,9 @@ public class MasterRpcServices extends RSRpcServices
 
       final byte[] regionName = req.getRegion().getValue().toByteArray();
       final RegionInfo regionInfo = master.getAssignmentManager().getRegionInfo(regionName);
-      if (regionInfo == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
+      if (regionInfo == null) {
+        throw new UnknownRegionException(Bytes.toStringBinary(regionName));
+      }
 
       final AssignRegionResponse arr = AssignRegionResponse.newBuilder().build();
       if (master.cpHost != null) {
@@ -668,7 +679,7 @@ public class MasterRpcServices extends RSRpcServices
 
   @Override
   public CreateTableResponse createTable(RpcController controller, CreateTableRequest req)
-  throws ServiceException {
+      throws ServiceException {
     TableDescriptor tableDescriptor = ProtobufUtil.toTableDescriptor(req.getTableSchema());
     byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
     try {
@@ -1062,7 +1073,7 @@ public class MasterRpcServices extends RSRpcServices
    * Get list of TableDescriptors for requested tables.
    * @param c Unused (set to null).
    * @param req GetTableDescriptorsRequest that contains:
-   * - tableNames: requested tables, or if empty, all are requested
+   *     - tableNames: requested tables, or if empty, all are requested.
    * @return GetTableDescriptorsResponse
    * @throws ServiceException
    */
@@ -1206,9 +1217,9 @@ public class MasterRpcServices extends RSRpcServices
   /**
    * Checks if the specified snapshot is done.
    * @return true if the snapshot is in file system ready to use,
-   *   false if the snapshot is in the process of completing
+   *     false if the snapshot is in the process of completing
    * @throws ServiceException wrapping UnknownSnapshotException if invalid snapshot, or
-   *  a wrapped HBaseSnapshotException with progress failure reason.
+   *     a wrapped HBaseSnapshotException with progress failure reason.
    */
   @Override
   public IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
@@ -1450,7 +1461,9 @@ public class MasterRpcServices extends RSRpcServices
 
       final byte[] regionName = request.getRegion().getValue().toByteArray();
       final RegionInfo hri = master.getAssignmentManager().getRegionInfo(regionName);
-      if (hri == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
+      if (hri == null) {
+        throw new UnknownRegionException(Bytes.toStringBinary(regionName));
+      }
 
       if (master.cpHost != null) {
         master.cpHost.preRegionOffline(hri);
@@ -2311,8 +2324,8 @@ public class MasterRpcServices extends RSRpcServices
             report.getRegionSize(), now);
         }
       } else {
-        LOG.debug(
-          "Received region space usage report but HMaster is not ready to process it, skipping");
+        LOG.debug("Received region space usage report but HMaster is not ready to process it, "
+            + "skipping");
       }
       return RegionSpaceUseReportResponse.newBuilder().build();
     } catch (Exception e) {
@@ -2348,8 +2361,8 @@ public class MasterRpcServices extends RSRpcServices
         }
         return builder.build();
       } else {
-        LOG.debug(
-          "Received space quota region size report but HMaster is not ready to process it, skipping");
+        LOG.debug("Received space quota region size report but HMaster is not ready to process it,"
+            + "skipping");
       }
       return builder.build();
     } catch (Exception e) {
@@ -2893,4 +2906,34 @@ public class MasterRpcServices extends RSRpcServices
     return true;
   }
 
+  @Override
+  public GetClusterIdResponse getClusterId(RpcController rpcController, GetClusterIdRequest request)
+      throws ServiceException {
+    GetClusterIdResponse.Builder resp = GetClusterIdResponse.newBuilder();
+    String clusterId = master.getClusterId();
+    if (clusterId != null) {
+      resp.setClusterId(clusterId);
+    }
+    return resp.build();
+  }
+
+  @Override
+  public GetActiveMasterResponse getActiveMaster(RpcController rpcController,
+      GetActiveMasterRequest request) throws ServiceException {
+    GetActiveMasterResponse.Builder resp = GetActiveMasterResponse.newBuilder();
+    Optional<ServerName> serverName = master.getActiveMaster();
+    serverName.ifPresent(name -> resp.setServerName(ProtobufUtil.toServerName(name)));
+    return resp.build();
+  }
+
+  @Override
+  public GetMetaRegionLocationsResponse getMetaRegionLocations(RpcController rpcController,
+      GetMetaRegionLocationsRequest request) throws ServiceException {
+    GetMetaRegionLocationsResponse.Builder response = GetMetaRegionLocationsResponse.newBuilder();
+    Optional<List<HRegionLocation>> metaLocations =
+        master.getMetaRegionLocationCache().getMetaRegionLocations();
+    metaLocations.ifPresent(hRegionLocations -> hRegionLocations.forEach(
+      location -> response.addMetaLocations(ProtobufUtil.toRegionLocation(location))));
+    return response.build();
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClientMetaServiceRPCs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClientMetaServiceRPCs.java
new file mode 100644
index 0000000..428aee2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClientMetaServiceRPCs.java
@@ -0,0 +1,164 @@
+/*
+ *
+ * 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 static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
+import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_TIMEOUT_KEY;
+import static org.junit.Assert.assertEquals;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.hadoop.hbase.ipc.RpcClientFactory;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClientMetaService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse;
+
+@Category({MediumTests.class, MasterTests.class})
+public class TestClientMetaServiceRPCs {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestClientMetaServiceRPCs.class);
+
+  // Total number of masters (active + stand by) for the purpose of this test.
+  private static final int MASTER_COUNT = 3;
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf;
+  private static int rpcTimeout;
+  private static RpcClient rpcClient;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    // Start the mini cluster with stand-by masters.
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    builder.numMasters(MASTER_COUNT).numRegionServers(3);
+    TEST_UTIL.startMiniCluster(builder.build());
+    conf = TEST_UTIL.getConfiguration();
+    rpcTimeout = (int) Math.min(Integer.MAX_VALUE, TimeUnit.MILLISECONDS.toNanos(
+        conf.getLong(HBASE_RPC_TIMEOUT_KEY, DEFAULT_HBASE_RPC_TIMEOUT)));
+    rpcClient = RpcClientFactory.createClient(conf,
+        TEST_UTIL.getMiniHBaseCluster().getMaster().getClusterId());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if (rpcClient != null) {
+      rpcClient.close();
+    }
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  private static ClientMetaService.BlockingInterface getMasterStub(ServerName server)
+      throws IOException {
+    return ClientMetaService.newBlockingStub(
+        rpcClient.createBlockingRpcChannel(server, User.getCurrent(), rpcTimeout));
+  }
+
+  private static HBaseRpcController getRpcController() {
+    return RpcControllerFactory.instantiate(conf).newController();
+  }
+
+  /**
+   * Verifies the cluster ID from all running masters.
+   */
+  @Test public void TestClusterID() throws Exception {
+    HBaseRpcController rpcController = getRpcController();
+    String clusterID = TEST_UTIL.getMiniHBaseCluster().getMaster().getClusterId();
+    int rpcCount = 0;
+    for (JVMClusterUtil.MasterThread masterThread:
+        TEST_UTIL.getMiniHBaseCluster().getMasterThreads()) {
+      ClientMetaService.BlockingInterface stub =
+          getMasterStub(masterThread.getMaster().getServerName());
+      GetClusterIdResponse resp =
+          stub.getClusterId(rpcController, GetClusterIdRequest.getDefaultInstance());
+      assertEquals(clusterID, resp.getClusterId());
+      rpcCount++;
+    }
+    assertEquals(MASTER_COUNT, rpcCount);
+  }
+
+  /**
+   * Verifies the active master ServerName as seen by all masters.
+   */
+  @Test public void TestActiveMaster() throws Exception {
+    HBaseRpcController rpcController = getRpcController();
+    ServerName activeMaster = TEST_UTIL.getMiniHBaseCluster().getMaster().getServerName();
+    int rpcCount = 0;
+    for (JVMClusterUtil.MasterThread masterThread:
+        TEST_UTIL.getMiniHBaseCluster().getMasterThreads()) {
+      ClientMetaService.BlockingInterface stub =
+          getMasterStub(masterThread.getMaster().getServerName());
+      GetActiveMasterResponse resp =
+          stub.getActiveMaster(rpcController, GetActiveMasterRequest.getDefaultInstance());
+      assertEquals(activeMaster, ProtobufUtil.toServerName(resp.getServerName()));
+      rpcCount++;
+    }
+    assertEquals(MASTER_COUNT, rpcCount);
+  }
+
+  /**
+   * Verifies that the meta region locations RPC returns consistent results across all masters.
+   */
+  @Test public void TestMetaLocations() throws Exception {
+    HBaseRpcController rpcController = getRpcController();
+    List<HRegionLocation> metaLocations = TEST_UTIL.getMiniHBaseCluster().getMaster()
+        .getMetaRegionLocationCache().getMetaRegionLocations().get();
+    Collections.sort(metaLocations);
+    int rpcCount = 0;
+    for (JVMClusterUtil.MasterThread masterThread:
+      TEST_UTIL.getMiniHBaseCluster().getMasterThreads()) {
+      ClientMetaService.BlockingInterface stub =
+          getMasterStub(masterThread.getMaster().getServerName());
+      GetMetaRegionLocationsResponse resp = stub.getMetaRegionLocations(
+          rpcController, GetMetaRegionLocationsRequest.getDefaultInstance());
+      List<HRegionLocation> result = new ArrayList<>();
+      resp.getMetaLocationsList().forEach(
+        location -> result.add(ProtobufUtil.toRegionLocation(location)));
+      Collections.sort(result);
+      assertEquals(metaLocations, result);
+      rpcCount++;
+    }
+    assertEquals(MASTER_COUNT, rpcCount);
+  }
+}


[hbase] 04/06: HBASE-23604: Clarify AsyncRegistry usage in the code. (#957)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e2a9f11f601f2f8c47d513cccc9102d13aaafba9
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Fri Jan 3 14:27:01 2020 -0800

    HBASE-23604: Clarify AsyncRegistry usage in the code. (#957)
    
    * HBASE-23604: Cleanup AsyncRegistry interface
    
    - Cleans up the method names to make more sense and adds a little
    more javadocs for context. In future patches we can revisit
    the name of the actual class to make it more self explanatory.
    
    - Does AsyncRegistry -> ConnectionRegistry rename.
    "async" ness of the registry is kind of implicit based on
    the interface contents and need not be reflected in the name.
    
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
    Signed-off-by: stack <st...@apache.org>
    Signed-off-by: Viraj Jasani <vj...@apache.org>
---
 .../hadoop/hbase/client/AsyncConnectionImpl.java   | 11 ++++++-----
 .../hbase/client/AsyncMetaRegionLocator.java       |  6 +++---
 .../hbase/client/AsyncTableRegionLocatorImpl.java  |  2 +-
 .../hadoop/hbase/client/ConnectionFactory.java     |  2 +-
 ...{AsyncRegistry.java => ConnectionRegistry.java} | 13 +++++++------
 ...Factory.java => ConnectionRegistryFactory.java} | 22 ++++++++++++----------
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    | 17 +++++++++--------
 ...syncRegistry.java => ZKConnectionRegistry.java} | 18 +++++++++---------
 ...istry.java => DoNothingConnectionRegistry.java} |  8 ++++----
 .../hbase/client/TestAsyncAdminRpcPriority.java    |  2 +-
 .../client/TestAsyncMetaRegionLocatorFailFast.java | 10 +++++-----
 .../hbase/client/TestAsyncTableRpcPriority.java    |  2 +-
 ...ryLeak.java => TestConnectionRegistryLeak.java} | 14 +++++++-------
 .../hbase/client/AsyncClusterConnectionImpl.java   |  4 ++--
 .../hbase/client/ClusterConnectionFactory.java     |  2 +-
 .../example/TestZooKeeperTableArchiveClient.java   | 18 +++++++++++-------
 .../hbase/client/AbstractTestRegionLocator.java    |  3 ++-
 ...cRegistry.java => DummyConnectionRegistry.java} | 11 ++++++-----
 .../hbase/client/RegionReplicaTestHelper.java      |  6 +++---
 .../client/TestAsyncAdminWithRegionReplicas.java   |  3 ++-
 .../hbase/client/TestAsyncMetaRegionLocator.java   |  4 ++--
 .../client/TestAsyncNonMetaRegionLocator.java      |  3 ++-
 ...stAsyncNonMetaRegionLocatorConcurrenyLimit.java |  3 ++-
 .../hbase/client/TestAsyncRegionLocator.java       |  3 ++-
 .../TestAsyncSingleRequestRpcRetryingCaller.java   |  3 ++-
 .../client/TestAsyncTableUseMetaReplicas.java      |  2 +-
 .../hbase/client/TestMetaRegionLocationCache.java  |  4 ++--
 ...Registry.java => TestZKConnectionRegistry.java} | 20 ++++++++++----------
 .../regionserver/TestWALEntrySinkFilter.java       | 19 ++++++++-----------
 29 files changed, 124 insertions(+), 111 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 78fad9e..9d90249 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -85,7 +85,7 @@ class AsyncConnectionImpl implements AsyncConnection {
 
   private final User user;
 
-  final AsyncRegistry registry;
+  final ConnectionRegistry registry;
 
   private final int rpcTimeout;
 
@@ -122,7 +122,7 @@ class AsyncConnectionImpl implements AsyncConnection {
 
   private volatile ConnectionOverAsyncConnection conn;
 
-  public AsyncConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId,
+  public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId,
       SocketAddress localAddress, User user) {
     this.conf = conf;
     this.user = user;
@@ -136,7 +136,8 @@ class AsyncConnectionImpl implements AsyncConnection {
     } else {
       this.metrics = Optional.empty();
     }
-    this.rpcClient = RpcClientFactory.createClient(conf, clusterId, localAddress, metrics.orElse(null));
+    this.rpcClient = RpcClientFactory.createClient(
+        conf, clusterId, localAddress, metrics.orElse(null));
     this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
     this.hostnameCanChange = conf.getBoolean(RESOLVE_HOSTNAME_ON_FAIL_KEY, true);
     this.rpcTimeout =
@@ -257,7 +258,7 @@ class AsyncConnectionImpl implements AsyncConnection {
   CompletableFuture<MasterService.Interface> getMasterStub() {
     return ConnectionUtils.getOrFetch(masterStub, masterStubMakeFuture, false, () -> {
       CompletableFuture<MasterService.Interface> future = new CompletableFuture<>();
-      addListener(registry.getMasterAddress(), (addr, error) -> {
+      addListener(registry.getActiveMaster(), (addr, error) -> {
         if (error != null) {
           future.completeExceptionally(error);
         } else if (addr == null) {
@@ -368,7 +369,7 @@ class AsyncConnectionImpl implements AsyncConnection {
   @Override
   public CompletableFuture<Hbck> getHbck() {
     CompletableFuture<Hbck> future = new CompletableFuture<>();
-    addListener(registry.getMasterAddress(), (sn, error) -> {
+    addListener(registry.getActiveMaster(), (sn, error) -> {
       if (error != null) {
         future.completeExceptionally(error);
       } else {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
index e920636..3571f960 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
@@ -38,14 +38,14 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 @InterfaceAudience.Private
 class AsyncMetaRegionLocator {
 
-  private final AsyncRegistry registry;
+  private final ConnectionRegistry registry;
 
   private final AtomicReference<RegionLocations> metaRegionLocations = new AtomicReference<>();
 
   private final AtomicReference<CompletableFuture<RegionLocations>> metaRelocateFuture =
     new AtomicReference<>();
 
-  AsyncMetaRegionLocator(AsyncRegistry registry) {
+  AsyncMetaRegionLocator(ConnectionRegistry registry) {
     this.registry = registry;
   }
 
@@ -60,7 +60,7 @@ class AsyncMetaRegionLocator {
    */
   CompletableFuture<RegionLocations> getRegionLocations(int replicaId, boolean reload) {
     return ConnectionUtils.getOrFetch(metaRegionLocations, metaRelocateFuture, reload,
-      registry::getMetaRegionLocation, locs -> isGood(locs, replicaId), "meta region location");
+      registry::getMetaRegionLocations, locs -> isGood(locs, replicaId), "meta region location");
   }
 
   private HRegionLocation getCacheLocation(HRegionLocation loc) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
index 0e116ab..fa3ea1c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
@@ -55,7 +55,7 @@ class AsyncTableRegionLocatorImpl implements AsyncTableRegionLocator {
   @Override
   public CompletableFuture<List<HRegionLocation>> getAllRegionLocations() {
     if (TableName.isMetaTableName(tableName)) {
-      return conn.registry.getMetaRegionLocation()
+      return conn.registry.getMetaRegionLocations()
         .thenApply(locs -> Arrays.asList(locs.getRegionLocations()));
     }
     return AsyncMetaTableAccessor.getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME),
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
index 2b432e4..027b898 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
@@ -279,7 +279,7 @@ public class ConnectionFactory {
   public static CompletableFuture<AsyncConnection> createAsyncConnection(Configuration conf,
       final User user) {
     CompletableFuture<AsyncConnection> future = new CompletableFuture<>();
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf);
+    ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf);
     addListener(registry.getClusterId(), (clusterId, error) -> {
       if (error != null) {
         registry.close();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java
similarity index 77%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java
index 9537777..cd22d78 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java
@@ -24,16 +24,17 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * Implementations hold cluster information such as this cluster's id, location of hbase:meta, etc..
+ * Registry for meta information needed for connection setup to a HBase cluster. Implementations
+ * hold cluster information such as this cluster's id, location of hbase:meta, etc..
  * Internal use only.
  */
 @InterfaceAudience.Private
-interface AsyncRegistry extends Closeable {
+interface ConnectionRegistry extends Closeable {
 
   /**
-   * Get the location of meta region.
+   * Get the location of meta region(s).
    */
-  CompletableFuture<RegionLocations> getMetaRegionLocation();
+  CompletableFuture<RegionLocations> getMetaRegionLocations();
 
   /**
    * Should only be called once.
@@ -43,9 +44,9 @@ interface AsyncRegistry extends Closeable {
   CompletableFuture<String> getClusterId();
 
   /**
-   * Get the address of HMaster.
+   * Get the address of active HMaster.
    */
-  CompletableFuture<ServerName> getMasterAddress();
+  CompletableFuture<ServerName> getActiveMaster();
 
   /**
    * Closes this instance and releases any system resources associated with it
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistryFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
similarity index 66%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistryFactory.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
index 28726ae..80d358b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistryFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.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
@@ -18,26 +18,28 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * Get instance of configured Registry.
+ * Factory class to get the instance of configured connection registry.
  */
 @InterfaceAudience.Private
-final class AsyncRegistryFactory {
+final class ConnectionRegistryFactory {
 
-  static final String REGISTRY_IMPL_CONF_KEY = "hbase.client.registry.impl";
+  static final String CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY =
+      "hbase.client.connection.registry.impl";
 
-  private AsyncRegistryFactory() {
+  private ConnectionRegistryFactory() {
   }
 
   /**
-   * @return The cluster registry implementation to use.
+   * @return The connection registry implementation to use.
    */
-  static AsyncRegistry getRegistry(Configuration conf) {
-    Class<? extends AsyncRegistry> clazz =
-        conf.getClass(REGISTRY_IMPL_CONF_KEY, ZKAsyncRegistry.class, AsyncRegistry.class);
+  static ConnectionRegistry getRegistry(Configuration conf) {
+    Class<? extends ConnectionRegistry> clazz = conf.getClass(
+        CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class,
+        ConnectionRegistry.class);
     return ReflectionUtils.newInstance(clazz, conf);
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 4a349d5..75324c9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -709,7 +709,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Boolean> isTableAvailable(TableName tableName) {
     if (TableName.isMetaTableName(tableName)) {
-      return connection.registry.getMetaRegionLocation().thenApply(locs -> Stream
+      return connection.registry.getMetaRegionLocations().thenApply(locs -> Stream
         .of(locs.getRegionLocations()).allMatch(loc -> loc != null && loc.getServerName() != null));
     }
     CompletableFuture<Boolean> future = new CompletableFuture<>();
@@ -847,7 +847,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<List<RegionInfo>> getRegions(TableName tableName) {
     if (tableName.equals(META_TABLE_NAME)) {
-      return connection.registry.getMetaRegionLocation()
+      return connection.registry.getMetaRegionLocations()
         .thenApply(locs -> Stream.of(locs.getRegionLocations()).map(HRegionLocation::getRegion)
           .collect(Collectors.toList()));
     } else {
@@ -1075,8 +1075,9 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     if (TableName.META_TABLE_NAME.equals(tableName)) {
       CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
       // For meta table, we use zk to fetch all locations.
-      AsyncRegistry registry = AsyncRegistryFactory.getRegistry(connection.getConfiguration());
-      addListener(registry.getMetaRegionLocation(), (metaRegions, err) -> {
+      ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(
+          connection.getConfiguration());
+      addListener(registry.getMetaRegionLocations(), (metaRegions, err) -> {
         if (err != null) {
           future.completeExceptionally(err);
         } else if (metaRegions == null || metaRegions.isEmpty() ||
@@ -1104,7 +1105,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
     switch (compactType) {
       case MOB:
-        addListener(connection.registry.getMasterAddress(), (serverName, err) -> {
+        addListener(connection.registry.getActiveMaster(), (serverName, err) -> {
           if (err != null) {
             future.completeExceptionally(err);
             return;
@@ -2343,7 +2344,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         String encodedName = Bytes.toString(regionNameOrEncodedRegionName);
         if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
           // old format encodedName, should be meta region
-          future = connection.registry.getMetaRegionLocation()
+          future = connection.registry.getMetaRegionLocations()
             .thenApply(locs -> Stream.of(locs.getRegionLocations())
               .filter(loc -> loc.getRegion().getEncodedName().equals(encodedName)).findFirst());
         } else {
@@ -2354,7 +2355,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         RegionInfo regionInfo =
           MetaTableAccessor.parseRegionInfoFromRegionName(regionNameOrEncodedRegionName);
         if (regionInfo.isMetaRegion()) {
-          future = connection.registry.getMetaRegionLocation()
+          future = connection.registry.getMetaRegionLocations()
             .thenApply(locs -> Stream.of(locs.getRegionLocations())
               .filter(loc -> loc.getRegion().getReplicaId() == regionInfo.getReplicaId())
               .findFirst());
@@ -2927,7 +2928,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
     switch (compactType) {
       case MOB:
-        addListener(connection.registry.getMasterAddress(), (serverName, err) -> {
+        addListener(connection.registry.getActiveMaster(), (serverName, err) -> {
           if (err != null) {
             future.completeExceptionally(err);
             return;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
similarity index 92%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
index 08e3846..3abfc9f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
@@ -50,15 +50,15 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
  * Zookeeper based registry implementation.
  */
 @InterfaceAudience.Private
-class ZKAsyncRegistry implements AsyncRegistry {
+class ZKConnectionRegistry implements ConnectionRegistry {
 
-  private static final Logger LOG = LoggerFactory.getLogger(ZKAsyncRegistry.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ZKConnectionRegistry.class);
 
   private final ReadOnlyZKClient zk;
 
   private final ZNodePaths znodePaths;
 
-  ZKAsyncRegistry(Configuration conf) {
+  ZKConnectionRegistry(Configuration conf) {
     this.znodePaths = new ZNodePaths(conf);
     this.zk = new ReadOnlyZKClient(conf);
   }
@@ -93,7 +93,7 @@ class ZKAsyncRegistry implements AsyncRegistry {
 
   @Override
   public CompletableFuture<String> getClusterId() {
-    return getAndConvert(znodePaths.clusterIdZNode, ZKAsyncRegistry::getClusterId);
+    return getAndConvert(znodePaths.clusterIdZNode, ZKConnectionRegistry::getClusterId);
   }
 
   @VisibleForTesting
@@ -144,7 +144,7 @@ class ZKAsyncRegistry implements AsyncRegistry {
       int replicaId = znodePaths.getMetaReplicaIdFromZnode(metaReplicaZNode);
       String path = ZNodePaths.joinZNode(znodePaths.baseZNode, metaReplicaZNode);
       if (replicaId == DEFAULT_REPLICA_ID) {
-        addListener(getAndConvert(path, ZKAsyncRegistry::getMetaProto), (proto, error) -> {
+        addListener(getAndConvert(path, ZKConnectionRegistry::getMetaProto), (proto, error) -> {
           if (error != null) {
             future.completeExceptionally(error);
             return;
@@ -162,7 +162,7 @@ class ZKAsyncRegistry implements AsyncRegistry {
           tryComplete(remaining, locs, future);
         });
       } else {
-        addListener(getAndConvert(path, ZKAsyncRegistry::getMetaProto), (proto, error) -> {
+        addListener(getAndConvert(path, ZKConnectionRegistry::getMetaProto), (proto, error) -> {
           if (future.isDone()) {
             return;
           }
@@ -191,7 +191,7 @@ class ZKAsyncRegistry implements AsyncRegistry {
   }
 
   @Override
-  public CompletableFuture<RegionLocations> getMetaRegionLocation() {
+  public CompletableFuture<RegionLocations> getMetaRegionLocations() {
     CompletableFuture<RegionLocations> future = new CompletableFuture<>();
     addListener(
       zk.list(znodePaths.baseZNode)
@@ -217,8 +217,8 @@ class ZKAsyncRegistry implements AsyncRegistry {
   }
 
   @Override
-  public CompletableFuture<ServerName> getMasterAddress() {
-    return getAndConvert(znodePaths.masterAddressZNode, ZKAsyncRegistry::getMasterProto)
+  public CompletableFuture<ServerName> getActiveMaster() {
+    return getAndConvert(znodePaths.masterAddressZNode, ZKConnectionRegistry::getMasterProto)
         .thenApply(proto -> {
           if (proto == null) {
             return null;
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingAsyncRegistry.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java
similarity index 89%
rename from hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingAsyncRegistry.java
rename to hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java
index 8c7b073..4bd6687 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingAsyncRegistry.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java
@@ -27,13 +27,13 @@ import org.apache.yetus.audience.InterfaceAudience;
  * Registry that does nothing. Otherwise, default Registry wants zookeeper up and running.
  */
 @InterfaceAudience.Private
-class DoNothingAsyncRegistry implements AsyncRegistry {
+class DoNothingConnectionRegistry implements ConnectionRegistry {
 
-  public DoNothingAsyncRegistry(Configuration conf) {
+  public DoNothingConnectionRegistry(Configuration conf) {
   }
 
   @Override
-  public CompletableFuture<RegionLocations> getMetaRegionLocation() {
+  public CompletableFuture<RegionLocations> getMetaRegionLocations() {
     return CompletableFuture.completedFuture(null);
   }
 
@@ -43,7 +43,7 @@ class DoNothingAsyncRegistry implements AsyncRegistry {
   }
 
   @Override
-  public CompletableFuture<ServerName> getMasterAddress() {
+  public CompletableFuture<ServerName> getActiveMaster() {
     return CompletableFuture.completedFuture(null);
   }
 
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java
index d4db7a4..75a9433 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java
@@ -142,7 +142,7 @@ public class TestAsyncAdminRpcPriority {
     }).when(adminStub).stopServer(any(HBaseRpcController.class), any(StopServerRequest.class),
       any());
 
-    conn = new AsyncConnectionImpl(CONF, new DoNothingAsyncRegistry(CONF), "test", null,
+    conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF), "test", null,
       UserProvider.instantiate(CONF).getCurrent()) {
 
       @Override
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java
index f29c3bf..b306500 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.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
@@ -43,21 +43,21 @@ public class TestAsyncMetaRegionLocatorFailFast {
 
   private static AsyncMetaRegionLocator LOCATOR;
 
-  private static final class FaultyAsyncRegistry extends DoNothingAsyncRegistry {
+  private static final class FaultyConnectionRegistry extends DoNothingConnectionRegistry {
 
-    public FaultyAsyncRegistry(Configuration conf) {
+    public FaultyConnectionRegistry(Configuration conf) {
       super(conf);
     }
 
     @Override
-    public CompletableFuture<RegionLocations> getMetaRegionLocation() {
+    public CompletableFuture<RegionLocations> getMetaRegionLocations() {
       return FutureUtils.failedFuture(new DoNotRetryRegionException("inject error"));
     }
   }
 
   @BeforeClass
   public static void setUp() {
-    LOCATOR = new AsyncMetaRegionLocator(new FaultyAsyncRegistry(CONF));
+    LOCATOR = new AsyncMetaRegionLocator(new FaultyConnectionRegistry(CONF));
   }
 
   @Test(expected = DoNotRetryIOException.class)
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java
index 15429cb..b104798 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java
@@ -175,7 +175,7 @@ public class TestAsyncTableRpcPriority {
         return null;
       }
     }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any());
-    conn = new AsyncConnectionImpl(CONF, new DoNothingAsyncRegistry(CONF), "test", null,
+    conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF), "test", null,
       UserProvider.instantiate(CONF).getCurrent()) {
 
       @Override
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegistryLeak.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java
similarity index 83%
rename from hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegistryLeak.java
rename to hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java
index 7c21075..f02ec42 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegistryLeak.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java
@@ -38,17 +38,17 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ ClientTests.class, SmallTests.class })
-public class TestAsyncRegistryLeak {
+public class TestConnectionRegistryLeak {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestAsyncRegistryLeak.class);
+    HBaseClassTestRule.forClass(TestConnectionRegistryLeak.class);
 
-  public static final class AsyncRegistryForTest extends DoNothingAsyncRegistry {
+  public static final class ConnectionRegistryForTest extends DoNothingConnectionRegistry {
 
     private boolean closed = false;
 
-    public AsyncRegistryForTest(Configuration conf) {
+    public ConnectionRegistryForTest(Configuration conf) {
       super(conf);
       CREATED.add(this);
     }
@@ -64,14 +64,14 @@ public class TestAsyncRegistryLeak {
     }
   }
 
-  private static final List<AsyncRegistryForTest> CREATED = new ArrayList<>();
+  private static final List<ConnectionRegistryForTest> CREATED = new ArrayList<>();
 
   private static Configuration CONF = HBaseConfiguration.create();
 
   @BeforeClass
   public static void setUp() {
-    CONF.setClass(AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY, AsyncRegistryForTest.class,
-      AsyncRegistry.class);
+    CONF.setClass(ConnectionRegistryFactory.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY,
+        ConnectionRegistryForTest.class, ConnectionRegistry.class);
   }
 
   @Test
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java
index 046ef41..f40f3ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java
@@ -50,8 +50,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpeci
 @InterfaceAudience.Private
 class AsyncClusterConnectionImpl extends AsyncConnectionImpl implements AsyncClusterConnection {
 
-  public AsyncClusterConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId,
-      SocketAddress localAddress, User user) {
+  public AsyncClusterConnectionImpl(Configuration conf, ConnectionRegistry registry,
+      String clusterId, SocketAddress localAddress, User user) {
     super(conf, registry, clusterId, localAddress, user);
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
index 46c0f5a..0094ffa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
@@ -49,7 +49,7 @@ public final class ClusterConnectionFactory {
    */
   public static AsyncClusterConnection createAsyncClusterConnection(Configuration conf,
       SocketAddress localAddress, User user) throws IOException {
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf);
+    ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf);
     String clusterId = FutureUtils.get(registry.getClusterId());
     Class<? extends AsyncClusterConnection> clazz =
       conf.getClass(HBASE_SERVER_CLUSTER_CONNECTION_IMPL, AsyncClusterConnectionImpl.class,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
index 0945bc9..e8aa822 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -40,7 +39,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.DummyAsyncRegistry;
+import org.apache.hadoop.hbase.client.DummyConnectionRegistry;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
 import org.apache.hadoop.hbase.master.cleaner.DirScanPool;
@@ -92,7 +91,7 @@ public class TestZooKeeperTableArchiveClient {
   private static RegionServerServices rss;
   private static DirScanPool POOL;
 
-  public static final class MockRegistry extends DummyAsyncRegistry {
+  public static final class MockRegistry extends DummyConnectionRegistry {
 
     public MockRegistry(Configuration conf) {
     }
@@ -110,8 +109,8 @@ public class TestZooKeeperTableArchiveClient {
   public static void setupCluster() throws Exception {
     setupConf(UTIL.getConfiguration());
     UTIL.startMiniZKCluster();
-    UTIL.getConfiguration().setClass("hbase.client.registry.impl", MockRegistry.class,
-      DummyAsyncRegistry.class);
+    UTIL.getConfiguration().setClass(MockRegistry.REGISTRY_IMPL_CONF_KEY, MockRegistry.class,
+      DummyConnectionRegistry.class);
     CONNECTION = ConnectionFactory.createConnection(UTIL.getConfiguration());
     archivingClient = new ZKTableArchiveClient(UTIL.getConfiguration(), CONNECTION);
     // make hfile archiving node so we can archive files
@@ -147,9 +146,13 @@ public class TestZooKeeperTableArchiveClient {
 
   @AfterClass
   public static void cleanupTest() throws Exception {
-    CONNECTION.close();
+    if (CONNECTION != null) {
+      CONNECTION.close();
+    }
     UTIL.shutdownMiniZKCluster();
-    POOL.shutdownNow();
+    if (POOL != null) {
+      POOL.shutdownNow();
+    }
   }
 
   /**
@@ -353,6 +356,7 @@ public class TestZooKeeperTableArchiveClient {
    * @throws IOException on failure
    * @throws KeeperException on failure
    */
+  @SuppressWarnings("checkstyle:EmptyBlock")
   private List<BaseHFileCleanerDelegate> turnOnArchiving(String tableName, HFileCleaner cleaner)
       throws IOException, KeeperException {
     // turn on hfile retention
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java
index 4668d15..f96daf6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java
@@ -58,7 +58,8 @@ public abstract class AbstractTestRegionLocator {
     }
     UTIL.getAdmin().createTable(td, SPLIT_KEYS);
     UTIL.waitTableAvailable(TABLE_NAME);
-    try (AsyncRegistry registry = AsyncRegistryFactory.getRegistry(UTIL.getConfiguration())) {
+    try (ConnectionRegistry registry =
+             ConnectionRegistryFactory.getRegistry(UTIL.getConfiguration())) {
       RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(UTIL.getConfiguration(),
         registry, REGION_REPLICATION);
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java
similarity index 81%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncRegistry.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java
index 245876e..a669362 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncRegistry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java
@@ -23,14 +23,15 @@ import org.apache.hadoop.hbase.ServerName;
 
 /**
  * Can be overridden in UT if you only want to implement part of the methods in
- * {@link AsyncRegistry}.
+ * {@link ConnectionRegistry}.
  */
-public class DummyAsyncRegistry implements AsyncRegistry {
+public class DummyConnectionRegistry implements ConnectionRegistry {
 
-  public static final String REGISTRY_IMPL_CONF_KEY = AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY;
+  public static final String REGISTRY_IMPL_CONF_KEY =
+      ConnectionRegistryFactory.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY;
 
   @Override
-  public CompletableFuture<RegionLocations> getMetaRegionLocation() {
+  public CompletableFuture<RegionLocations> getMetaRegionLocations() {
     return null;
   }
 
@@ -40,7 +41,7 @@ public class DummyAsyncRegistry implements AsyncRegistry {
   }
 
   @Override
-  public CompletableFuture<ServerName> getMasterAddress() {
+  public CompletableFuture<ServerName> getActiveMaster() {
     return null;
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java
index ba3e74c..73924a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java
@@ -44,7 +44,7 @@ public final class RegionReplicaTestHelper {
 
   // waits for all replicas to have region location
   static void waitUntilAllMetaReplicasHavingRegionLocation(Configuration conf,
-      AsyncRegistry registry, int regionReplication) throws IOException {
+      ConnectionRegistry registry, int regionReplication) throws IOException {
     Waiter.waitFor(conf, conf.getLong("hbase.client.sync.wait.timeout.msec", 60000), 200, true,
       new ExplainingPredicate<IOException>() {
         @Override
@@ -55,7 +55,7 @@ public final class RegionReplicaTestHelper {
         @Override
         public boolean evaluate() throws IOException {
           try {
-            RegionLocations locs = registry.getMetaRegionLocation().get();
+            RegionLocations locs = registry.getMetaRegionLocations().get();
             if (locs.size() < regionReplication) {
               return false;
             }
@@ -66,7 +66,7 @@ public final class RegionReplicaTestHelper {
             }
             return true;
           } catch (Exception e) {
-            TestZKAsyncRegistry.LOG.warn("Failed to get meta region locations", e);
+            TestZKConnectionRegistry.LOG.warn("Failed to get meta region locations", e);
             return false;
           }
         }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java
index 1401afe..6d7d368 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java
@@ -53,7 +53,8 @@ public class TestAsyncAdminWithRegionReplicas extends TestAsyncAdminBase {
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
     TestAsyncAdminBase.setUpBeforeClass();
-    try (AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration())) {
+    try (ConnectionRegistry registry =
+             ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration())) {
       RegionReplicaTestHelper
         .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), registry, 3);
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
index 1f0d40b..609a129 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
@@ -45,7 +45,7 @@ public class TestAsyncMetaRegionLocator {
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
-  private static AsyncRegistry REGISTRY;
+  private static ConnectionRegistry REGISTRY;
 
   private static AsyncMetaRegionLocator LOCATOR;
 
@@ -54,7 +54,7 @@ public class TestAsyncMetaRegionLocator {
     TEST_UTIL.getConfiguration().set(BaseLoadBalancer.TABLES_ON_MASTER, "none");
     TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
     TEST_UTIL.startMiniCluster(3);
-    REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     RegionReplicaTestHelper
       .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), REGISTRY, 3);
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
index d1ed5b7..d8388de 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
@@ -79,7 +79,8 @@ public class TestAsyncNonMetaRegionLocator {
   public static void setUp() throws Exception {
     TEST_UTIL.startMiniCluster(3);
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    ConnectionRegistry registry =
+        ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
       registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = new AsyncNonMetaRegionLocator(CONN);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
index 7e06218..88ab3ad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
@@ -123,7 +123,8 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
     conf.setInt(MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE, MAX_ALLOWED);
     TEST_UTIL.startMiniCluster(3);
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    ConnectionRegistry registry =
+        ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
       registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = new AsyncNonMetaRegionLocator(CONN);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java
index a7cf322..753e4f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java
@@ -98,7 +98,8 @@ public class TestAsyncRegionLocator {
     TEST_UTIL.startMiniCluster(1);
     TEST_UTIL.createTable(TABLE_NAME, FAMILY);
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    ConnectionRegistry registry =
+        ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
       registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = CONN.getLocator();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
index b6e5362..4205012 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
@@ -71,7 +71,8 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
     TEST_UTIL.createTable(TABLE_NAME, FAMILY);
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    ConnectionRegistry registry =
+        ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
       registry.getClusterId().get(), null, User.getCurrent());
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java
index e74d3fa..ed6c66f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java
@@ -92,7 +92,7 @@ public class TestAsyncTableUseMetaReplicas {
     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
       FailPrimaryMetaScanCp.class.getName());
     UTIL.startMiniCluster(3);
-    try (AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf)) {
+    try (ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf)) {
       RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(conf, registry, 3);
     }
     try (Table table = UTIL.createTable(TABLE_NAME, FAMILY)) {
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
index 02236a6..1205b05 100644
--- 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
@@ -53,13 +53,13 @@ public class TestMetaRegionLocationCache {
       HBaseClassTestRule.forClass(TestMetaRegionLocationCache.class);
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static AsyncRegistry REGISTRY;
+  private static ConnectionRegistry 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());
+    REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(
         TEST_UTIL.getConfiguration(), REGISTRY, 3);
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java
similarity index 89%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java
index 3e4ca94..cb45ee5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java
@@ -48,16 +48,16 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @Category({ MediumTests.class, ClientTests.class })
-public class TestZKAsyncRegistry {
+public class TestZKConnectionRegistry {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestZKAsyncRegistry.class);
+    HBaseClassTestRule.forClass(TestZKConnectionRegistry.class);
 
-  static final Logger LOG = LoggerFactory.getLogger(TestZKAsyncRegistry.class);
+  static final Logger LOG = LoggerFactory.getLogger(TestZKConnectionRegistry.class);
   static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
-  private static ZKAsyncRegistry REGISTRY;
+  private static ZKConnectionRegistry REGISTRY;
 
   @BeforeClass
   public static void setUp() throws Exception {
@@ -67,7 +67,7 @@ public class TestZKAsyncRegistry {
     // make sure that we do not depend on this config when getting locations for meta replicas, see
     // HBASE-21658.
     conf.setInt(META_REPLICAS_NUM, 1);
-    REGISTRY = new ZKAsyncRegistry(conf);
+    REGISTRY = new ZKConnectionRegistry(conf);
   }
 
   @AfterClass
@@ -84,10 +84,10 @@ public class TestZKAsyncRegistry {
     assertEquals("Expected " + expectedClusterId + ", found=" + clusterId, expectedClusterId,
       clusterId);
     assertEquals(TEST_UTIL.getHBaseCluster().getMaster().getServerName(),
-      REGISTRY.getMasterAddress().get());
+      REGISTRY.getActiveMaster().get());
     RegionReplicaTestHelper
       .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), REGISTRY, 3);
-    RegionLocations locs = REGISTRY.getMetaRegionLocation().get();
+    RegionLocations locs = REGISTRY.getMetaRegionLocations().get();
     assertEquals(3, locs.getRegionLocations().length);
     IntStream.range(0, 3).forEach(i -> {
       HRegionLocation loc = locs.getRegionLocation(i);
@@ -102,7 +102,7 @@ public class TestZKAsyncRegistry {
     try (ReadOnlyZKClient zk1 = REGISTRY.getZKClient()) {
       Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration());
       otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1");
-      try (ZKAsyncRegistry otherRegistry = new ZKAsyncRegistry(otherConf)) {
+      try (ZKConnectionRegistry otherRegistry = new ZKConnectionRegistry(otherConf)) {
         ReadOnlyZKClient zk2 = otherRegistry.getZKClient();
         assertNotSame("Using a different configuration / quorum should result in different " +
           "backing zk connection.", zk1, zk2);
@@ -119,9 +119,9 @@ public class TestZKAsyncRegistry {
   public void testNoMetaAvailable() throws InterruptedException {
     Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
     conf.set("zookeeper.znode.metaserver", "whatever");
-    try (ZKAsyncRegistry registry = new ZKAsyncRegistry(conf)) {
+    try (ZKConnectionRegistry registry = new ZKConnectionRegistry(conf)) {
       try {
-        registry.getMetaRegionLocation().get();
+        registry.getMetaRegionLocations().get();
         fail("Should have failed since we set an incorrect meta znode prefix");
       } catch (ExecutionException e) {
         assertThat(e.getCause(), instanceOf(IOException.class));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java
index 41d4f46..5e16723 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.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
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.replication.regionserver;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.net.SocketAddress;
 import java.util.ArrayList;
@@ -43,8 +42,8 @@ import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.AsyncTable;
 import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.DummyAsyncClusterConnection;
-import org.apache.hadoop.hbase.client.DummyAsyncRegistry;
 import org.apache.hadoop.hbase.client.DummyAsyncTable;
+import org.apache.hadoop.hbase.client.DummyConnectionRegistry;
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -57,9 +56,7 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
-
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 
 /**
@@ -108,12 +105,12 @@ public class TestWALEntrySinkFilter {
   public void testWALEntryFilter() throws IOException {
     Configuration conf = HBaseConfiguration.create();
     // Make it so our filter is instantiated on construction of ReplicationSink.
-    conf.setClass(DummyAsyncRegistry.REGISTRY_IMPL_CONF_KEY, DevNullAsyncRegistry.class,
-      DummyAsyncRegistry.class);
+    conf.setClass(DummyConnectionRegistry.REGISTRY_IMPL_CONF_KEY, DevNullConnectionRegistry.class,
+        DummyConnectionRegistry.class);
     conf.setClass(WALEntrySinkFilter.WAL_ENTRY_FILTER_KEY,
-      IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl.class, WALEntrySinkFilter.class);
+        IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl.class, WALEntrySinkFilter.class);
     conf.setClass(ClusterConnectionFactory.HBASE_SERVER_CLUSTER_CONNECTION_IMPL,
-      DevNullAsyncClusterConnection.class, AsyncClusterConnection.class);
+        DevNullAsyncClusterConnection.class, AsyncClusterConnection.class);
     ReplicationSink sink = new ReplicationSink(conf, STOPPABLE);
     // Create some dumb walentries.
     List<AdminProtos.WALEntry> entries = new ArrayList<>();
@@ -190,9 +187,9 @@ public class TestWALEntrySinkFilter {
     }
   }
 
-  public static class DevNullAsyncRegistry extends DummyAsyncRegistry {
+  public static class DevNullConnectionRegistry extends DummyConnectionRegistry {
 
-    public DevNullAsyncRegistry(Configuration conf) {
+    public DevNullConnectionRegistry(Configuration conf) {
     }
 
     @Override