You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2019/03/03 06:06:23 UTC

[hbase] branch HBASE-21512 updated (b10ab4a -> e4bded6)

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

zhangduo pushed a change to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git.


 discard b10ab4a  HBASE-21585 Remove ClusterConnection
 discard d109fb7  HBASE-21779 Reimplement BulkLoadHFilesTool to use AsyncClusterConnection
 discard 9c900d7d HBASE-21778 Remove the usage of the locateRegion related methods in ClusterConnection
 discard 53fce97  HBASE-21719 Rewrite RegionPlacementMaintainer to use AsyncClusterConnection
 discard 8a8676e3 HBASE-21537 Rewrite ServerManager.closeRegionSilentlyAndWait to use AsyncClusterConnection
 discard c1762d7  HBASE-21671 Rewrite RegionReplicaReplicationEndpoint to use AsyncClusterConnection
 discard 363e508  HBASE-21538 Rewrite RegionReplicaFlushHandler to use AsyncClusterConnection
 discard e464bf9  HBASE-21579 Use AsyncClusterConnection for HBaseInterClusterReplicationEndpoint
 discard 412ad01  HBASE-21526 Use AsyncClusterConnection in ServerManager for getRsAdmin
 discard 4326f20  HBASE-21516 Use AsyncConnection instead of Connection in SecureBulkLoadManager
 discard 0073060  HBASE-21515 Also initialize an AsyncClusterConnection in HRegionServer
     add 9370347  HBASE-21820 Implement CLUSTER quota scope
     add e65744a  HBASE-21450 [documentation] Point spark doc at hbase-connectors spark
     add c4f5d3c  HBASE-21967 Split TestServerCrashProcedure and TestServerCrashProcedureWithReplicas
     add c19bc59  HBASE-21934 RemoteProcedureDispatcher should track the ongoing dispatched calls
     add d725d33  HBASE-21969 Improve the update of destination rsgroup of RSGroupInfoManagerImpl#moveTables()
     add fd152c2  HBASE-21952 Fix unit test: TestClientOperationInterrupt#testInterrupt50Percent
     add 929a8aa  HBASE-21082 Reimplement assign/unassign related procedure metrics
     add f748e48  HBASE-21481 [acl] Superuser's permissions should not be granted or revoked by any non-su global admin
     add cbdbe65  HBASE-21976 Deal with RetryImmediatelyException for batching request
     add 030b4d1  HBASE-21487 Concurrent modify table ops can lead to unexpected results
     add f8e84ca  HBASE-21970 Document that how to upgrade from 2.0 or 2.1 to 2.2+
     add 4e1f396  HBASE-21898 Update downloads page with 1.2.11 release.
     add 841a45a  HBASE-21974 Change Admin#grant/revoke parameter from UserPermission to user and Permission
     add 1e12e88  HBASE-21978 Should close AsyncRegistry if we fail to get cluster id when creating AsyncConnection
     add 7142cea  HBASE-21980 Fix typo in AbstractTestAsyncTableRegionReplicasRead
     new 713da50  HBASE-21515 Also initialize an AsyncClusterConnection in HRegionServer
     new 5564ddd  HBASE-21516 Use AsyncConnection instead of Connection in SecureBulkLoadManager
     new 1ee798a  HBASE-21526 Use AsyncClusterConnection in ServerManager for getRsAdmin
     new 247a3b8  HBASE-21579 Use AsyncClusterConnection for HBaseInterClusterReplicationEndpoint
     new 6220997  HBASE-21538 Rewrite RegionReplicaFlushHandler to use AsyncClusterConnection
     new f83e7af  HBASE-21671 Rewrite RegionReplicaReplicationEndpoint to use AsyncClusterConnection
     new afbb50d  HBASE-21537 Rewrite ServerManager.closeRegionSilentlyAndWait to use AsyncClusterConnection
     new 64ea281  HBASE-21719 Rewrite RegionPlacementMaintainer to use AsyncClusterConnection
     new 69469f3  HBASE-21778 Remove the usage of the locateRegion related methods in ClusterConnection
     new c1ead82  HBASE-21779 Reimplement BulkLoadHFilesTool to use AsyncClusterConnection
     new e4bded6  HBASE-21585 Remove ClusterConnection

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   (b10ab4a)
            \
             N -- N -- N   refs/heads/HBASE-21512 (e4bded6)

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 11 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:
 ...a => ConcurrentTableModificationException.java} |  26 +-
 .../java/org/apache/hadoop/hbase/client/Admin.java |  15 +-
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java |  13 +-
 .../hbase/client/AsyncBatchRpcRetryingCaller.java  |  30 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       |  10 +-
 .../hadoop/hbase/client/ConnectionFactory.java     |   4 +-
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java |  12 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |  20 +-
 .../hadoop/hbase/quotas/QuotaSettingsFactory.java  | 100 ++++-
 .../hbase/security/access/AccessControlClient.java |  26 +-
 .../hbase/security/access/AccessControlUtil.java   |  12 +-
 .../hadoop/hbase/security/access/Permission.java   |  75 ++++
 ...torFailFast.java => TestAsyncRegistryLeak.java} |  53 ++-
 .../apache/hadoop/hbase/security/Superusers.java   |  17 +-
 .../org/apache/hadoop/hbase/security/User.java     |   9 +-
 .../apache/hadoop/hbase/security/UserProvider.java |  10 +
 .../master/MetricsAssignmentManagerSource.java     |  28 +-
 .../master/MetricsAssignmentManagerSourceImpl.java |  28 ++
 .../procedure2/RemoteProcedureDispatcher.java      |  29 ++
 .../src/main/protobuf/MasterProcedure.proto        |  16 +-
 .../hbase/rsgroup/RSGroupInfoManagerImpl.java      |  20 +-
 .../org/apache/hadoop/hbase/master/HMaster.java    |  16 +-
 .../hbase/master/MetricsAssignmentManager.java     |  36 ++
 .../master/assignment/CloseRegionProcedure.java    |   6 +
 .../master/assignment/OpenRegionProcedure.java     |   6 +
 .../assignment/RegionRemoteProcedureBase.java      |   5 +
 .../assignment/RegionTransitionProcedure.java      |   1 +
 .../assignment/TransitRegionStateProcedure.java    | 105 +++--
 .../master/procedure/ModifyTableProcedure.java     |  48 ++-
 .../master/procedure/ServerRemoteProcedure.java    | 131 ++++++
 .../master/procedure/SplitWALRemoteProcedure.java  |  84 +---
 .../SwitchRpcThrottleRemoteProcedure.java          |  61 +--
 .../master/replication/RefreshPeerProcedure.java   |  71 +--
 .../SyncReplicationReplayWALRemoteProcedure.java   |  70 +--
 .../org/apache/hadoop/hbase/quotas/QuotaCache.java |  57 ++-
 .../org/apache/hadoop/hbase/quotas/QuotaUtil.java  |  84 +++-
 .../hbase/security/access/AccessChecker.java       |  49 ++-
 .../hbase/security/access/AccessController.java    |  12 +-
 .../hadoop/hbase/security/access/AuthManager.java  |  57 +--
 .../AbstractTestAsyncTableRegionReplicasRead.java  |   2 +-
 .../TestAsyncTableBatchRetryImmediately.java       | 101 +++++
 .../hbase/client/TestClientOperationInterrupt.java |   7 +-
 .../master/assignment/TestAssignmentManager.java   |  44 ++
 .../assignment/TestAssignmentManagerBase.java      |  26 +-
 .../hbase/master/assignment/TestRegionBypass.java  |  18 +-
 .../master/procedure/TestModifyTableProcedure.java | 178 ++++++++
 ...{TestServerCrashProcedure.java => TestSCP.java} |  43 +-
 ...eWithReplicas.java => TestSCPWithReplicas.java} |   8 +-
 .../TestSCPWithReplicasWithoutZKCoordinated.java}  |  13 +-
 .../TestSCPWithoutZKCoordinated.java}              |  15 +-
 .../procedure/TestServerRemoteProcedure.java       | 282 ++++++++++++
 .../quotas/TestClusterScopeQuotaThrottle.java      | 236 ++++++++++
 .../apache/hadoop/hbase/quotas/TestQuotaAdmin.java |  63 ++-
 .../hadoop/hbase/quotas/TestQuotaThrottle.java     | 478 ++++++++-------------
 .../hadoop/hbase/quotas/ThrottleQuotaTestUtil.java | 207 +++++++++
 .../hbase/security/access/SecureTestUtil.java      | 143 +++++-
 .../security/access/TestAccessController.java      |  11 +-
 .../security/access/TestNamespaceCommands.java     |  18 +-
 .../security/access/TestPermissionBuilder.java     | 125 ++++++
 .../hbase/security/access/TestRpcAccessChecks.java |  99 ++++-
 hbase-shell/src/main/ruby/hbase/quotas.rb          |  26 +-
 .../src/main/ruby/shell/commands/set_quota.rb      |  27 +-
 hbase-shell/src/test/ruby/hbase/quotas_test.rb     |  17 +
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   |   6 +-
 src/main/asciidoc/_chapters/spark.adoc             |  57 +--
 src/main/asciidoc/_chapters/upgrading.adoc         |  14 +
 src/site/xdoc/downloads.xml                        |  14 +-
 67 files changed, 2790 insertions(+), 940 deletions(-)
 copy hbase-client/src/main/java/org/apache/hadoop/hbase/{TableNotEnabledException.java => ConcurrentTableModificationException.java} (67%)
 copy hbase-client/src/test/java/org/apache/hadoop/hbase/client/{TestAsyncMetaRegionLocatorFailFast.java => TestAsyncRegistryLeak.java} (57%)
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerRemoteProcedure.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatchRetryImmediately.java
 rename hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/{TestServerCrashProcedure.java => TestSCP.java} (86%)
 rename hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/{TestServerCrashProcedureWithReplicas.java => TestSCPWithReplicas.java} (92%)
 copy hbase-server/src/test/java/org/apache/hadoop/hbase/master/{TestDLSAsyncFSWAL.java => procedure/TestSCPWithReplicasWithoutZKCoordinated.java} (72%)
 copy hbase-server/src/test/java/org/apache/hadoop/hbase/master/{TestDLSAsyncFSWAL.java => procedure/TestSCPWithoutZKCoordinated.java} (71%)
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestClusterScopeQuotaThrottle.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/ThrottleQuotaTestUtil.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestPermissionBuilder.java


[hbase] 09/11: HBASE-21778 Remove the usage of the locateRegion related methods in ClusterConnection

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

zhangduo pushed a commit to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 69469f3ff930bc9babd52a93509c5d96561606fc
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Fri Feb 1 16:40:34 2019 +0800

    HBASE-21778 Remove the usage of the locateRegion related methods in ClusterConnection
    
    Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../apache/hadoop/hbase/client/AsyncProcess.java   |   4 +-
 .../hadoop/hbase/client/BufferedMutatorImpl.java   |   3 +-
 .../hbase/client/ClientAsyncPrefetchScanner.java   |   4 +-
 .../apache/hadoop/hbase/client/ClientScanner.java  |   6 +-
 .../hadoop/hbase/client/ClientServiceCallable.java |   4 +-
 .../hadoop/hbase/client/ClientSimpleScanner.java   |   4 +-
 .../hadoop/hbase/client/ClusterConnection.java     | 149 ---------------------
 .../hbase/client/ConnectionImplementation.java     | 127 ++++++++++++------
 .../hadoop/hbase/client/FlushRegionCallable.java   |   6 +-
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java |  10 +-
 .../org/apache/hadoop/hbase/client/HTable.java     |   4 +-
 .../hbase/client/RegionAdminServiceCallable.java   |  12 +-
 .../hadoop/hbase/client/RegionServerCallable.java  |  10 +-
 .../hadoop/hbase/client/ReversedClientScanner.java |   4 +-
 .../client/RpcRetryingCallerWithReadReplicas.java  |  36 +++--
 .../hbase/client/ScannerCallableWithReplicas.java  |  27 ++--
 .../hadoop/hbase/client/TestAsyncProcess.java      |  98 +++++++-------
 .../TestAsyncProcessWithRegionException.java       |  10 +-
 .../hadoop/hbase/client/TestBufferedMutator.java   |   2 +-
 .../hadoop/hbase/client/TestClientScanner.java     |   8 +-
 .../hbase/client/TestReversedScannerCallable.java  |   3 +-
 ...nTestTimeBoundedRequestsWithRegionReplicas.java |  13 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java    |  34 ++---
 .../apache/hadoop/hbase/util/RegionSplitter.java   |  19 ++-
 .../hbase/client/HConnectionTestingUtility.java    | 105 +++++++--------
 .../apache/hadoop/hbase/client/TestCISleep.java    |   4 +-
 .../hbase/client/TestHBaseAdminNoCluster.java      |   7 +-
 .../TestMetaTableAccessorNoCluster.java            |  47 ++++---
 .../hadoop/hbase/client/TestMetaWithReplicas.java  |  34 ++---
 .../hbase/client/TestReplicaWithCluster.java       |  49 ++++---
 .../hadoop/hbase/client/TestReplicasClient.java    |  11 +-
 .../hbase/client/TestSeparateClientZKCluster.java  |   4 +-
 .../client/TestSnapshotCloneIndependence.java      |   2 +-
 .../master/TestMetaAssignmentWithStopMaster.java   |  48 +++----
 .../TestLoadIncrementalHFilesSplitRecovery.java    |  27 ++--
 .../hadoop/hbase/util/BaseTestHBaseFsck.java       |   3 +-
 .../hadoop/hbase/util/MultiThreadedAction.java     |  23 ++--
 .../hadoop/hbase/util/MultiThreadedReader.java     |  11 +-
 .../hadoop/hbase/util/MultiThreadedWriterBase.java |   8 +-
 39 files changed, 431 insertions(+), 549 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
index de7449b..b0f863f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
@@ -136,7 +136,7 @@ class AsyncProcess {
   // TODO: many of the fields should be made private
   final long id;
 
-  final ClusterConnection connection;
+  final ConnectionImplementation connection;
   private final RpcRetryingCallerFactory rpcCallerFactory;
   final RpcControllerFactory rpcFactory;
 
@@ -161,7 +161,7 @@ class AsyncProcess {
   public static final String LOG_DETAILS_PERIOD = "hbase.client.log.detail.period.ms";
   private static final int DEFAULT_LOG_DETAILS_PERIOD = 10000;
   private final int periodToLog;
-  AsyncProcess(ClusterConnection hc, Configuration conf,
+  AsyncProcess(ConnectionImplementation hc, Configuration conf,
       RpcRetryingCallerFactory rpcCaller, RpcControllerFactory rpcFactory) {
     if (hc == null) {
       throw new IllegalArgumentException("ClusterConnection cannot be null.");
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
index f0c8da4..922611b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
@@ -140,7 +140,8 @@ public class BufferedMutatorImpl implements BufferedMutator {
             params.getOperationTimeout() : conn.getConnectionConfiguration().getOperationTimeout());
     this.ap = ap;
   }
-  BufferedMutatorImpl(ClusterConnection conn, RpcRetryingCallerFactory rpcCallerFactory,
+
+  BufferedMutatorImpl(ConnectionImplementation conn, RpcRetryingCallerFactory rpcCallerFactory,
       RpcControllerFactory rpcFactory, BufferedMutatorParams params) {
     this(conn, params,
       // puts need to track errors globally due to how the APIs currently work.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
index e5af871..1a9178c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
@@ -67,11 +67,11 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
   private final Condition notFull = lock.newCondition();
 
   public ClientAsyncPrefetchScanner(Configuration configuration, Scan scan, TableName name,
-      ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
+      ConnectionImplementation connection, RpcRetryingCallerFactory rpcCallerFactory,
       RpcControllerFactory rpcControllerFactory, ExecutorService pool,
       int replicaCallTimeoutMicroSecondScan) throws IOException {
     super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool,
-        replicaCallTimeoutMicroSecondScan);
+      replicaCallTimeoutMicroSecondScan);
   }
 
   @VisibleForTesting
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
index cae98aa..bbc3fb4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
@@ -68,7 +68,7 @@ public abstract class ClientScanner extends AbstractClientScanner {
   // Keep lastResult returned successfully in case we have to reset scanner.
   protected Result lastResult = null;
   protected final long maxScannerResultSize;
-  private final ClusterConnection connection;
+  private final ConnectionImplementation connection;
   protected final TableName tableName;
   protected final int scannerTimeout;
   protected RpcRetryingCaller<Result[]> caller;
@@ -93,7 +93,7 @@ public abstract class ClientScanner extends AbstractClientScanner {
    * @throws IOException
    */
   public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName,
-      ClusterConnection connection, RpcRetryingCallerFactory rpcFactory,
+      ConnectionImplementation connection, RpcRetryingCallerFactory rpcFactory,
       RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout)
       throws IOException {
     if (LOG.isTraceEnabled()) {
@@ -137,7 +137,7 @@ public abstract class ClientScanner extends AbstractClientScanner {
     initCache();
   }
 
-  protected ClusterConnection getConnection() {
+  protected ConnectionImplementation getConnection() {
     return this.connection;
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java
index f118e7a..67ba838 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java
@@ -34,9 +34,9 @@ import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 public abstract class ClientServiceCallable<T> extends
     RegionServerCallable<T, ClientProtos.ClientService.BlockingInterface> {
 
-  public ClientServiceCallable(Connection connection, TableName tableName, byte [] row,
+  public ClientServiceCallable(Connection connection, TableName tableName, byte[] row,
       RpcController rpcController, int priority) {
-    super(connection, tableName, row, rpcController, priority);
+    super((ConnectionImplementation) connection, tableName, row, rpcController, priority);
   }
 
   @Override
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
index 7e9c4b9..e5d7b97 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
@@ -37,11 +37,11 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 @InterfaceAudience.Private
 public class ClientSimpleScanner extends ClientScanner {
   public ClientSimpleScanner(Configuration configuration, Scan scan, TableName name,
-      ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
+      ConnectionImplementation connection, RpcRetryingCallerFactory rpcCallerFactory,
       RpcControllerFactory rpcControllerFactory, ExecutorService pool,
       int replicaCallTimeoutMicroSecondScan) throws IOException {
     super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool,
-        replicaCallTimeoutMicroSecondScan);
+      replicaCallTimeoutMicroSecondScan);
   }
 
   @Override
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
index 9b2222b..7828ef0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
@@ -20,11 +20,8 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
-import java.util.List;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MasterNotRunningException;
-import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@@ -95,135 +92,6 @@ public interface ClusterConnection extends Connection {
   TableState getTableState(TableName tableName)  throws IOException;
 
   /**
-   * Find the location of the region of <i>tableName</i> that <i>row</i>
-   * lives in.
-   * @param tableName name of the table <i>row</i> is in
-   * @param row row key you're trying to find the region of
-   * @return HRegionLocation that describes where to find the region in
-   *   question
-   * @throws IOException if a remote or network exception occurs
-   */
-  HRegionLocation locateRegion(final TableName tableName,
-      final byte [] row) throws IOException;
-
-  /**
-   * @deprecated {@link #clearRegionLocationCache()} instead.
-   */
-  @Deprecated
-  default void clearRegionCache() {
-    clearRegionLocationCache();
-  }
-
-  void cacheLocation(final TableName tableName, final RegionLocations location);
-
-  /**
-   * Allows flushing the region cache of all locations that pertain to
-   * <code>tableName</code>
-   * @param tableName Name of the table whose regions we are to remove from
-   *   cache.
-   */
-  void clearRegionCache(final TableName tableName);
-
-  /**
-   * Deletes cached locations for the specific region.
-   * @param location The location object for the region, to be purged from cache.
-   */
-  void deleteCachedRegionLocation(final HRegionLocation location);
-
-  /**
-   * Find the location of the region of <i>tableName</i> that <i>row</i>
-   * lives in, ignoring any value that might be in the cache.
-   * @param tableName name of the table <i>row</i> is in
-   * @param row row key you're trying to find the region of
-   * @return HRegionLocation that describes where to find the region in
-   *   question
-   * @throws IOException if a remote or network exception occurs
-   */
-  HRegionLocation relocateRegion(final TableName tableName,
-      final byte [] row) throws IOException;
-
-  /**
-   * Find the location of the region of <i>tableName</i> that <i>row</i>
-   * lives in, ignoring any value that might be in the cache.
-   * @param tableName name of the table <i>row</i> is in
-   * @param row row key you're trying to find the region of
-   * @param replicaId the replicaId of the region
-   * @return RegionLocations that describe where to find the region in
-   *   question
-   * @throws IOException if a remote or network exception occurs
-   */
-  RegionLocations relocateRegion(final TableName tableName,
-      final byte [] row, int replicaId) throws IOException;
-
-  /**
-   * Update the location cache. This is used internally by HBase, in most cases it should not be
-   *  used by the client application.
-   * @param tableName the table name
-   * @param regionName the region name
-   * @param rowkey the row
-   * @param exception the exception if any. Can be null.
-   * @param source the previous location
-   */
-  void updateCachedLocations(TableName tableName, byte[] regionName, byte[] rowkey,
-                                    Object exception, ServerName source);
-
-  /**
-   * Gets the location of the region of <i>regionName</i>.
-   * @param regionName name of the region to locate
-   * @return HRegionLocation that describes where to find the region in
-   *   question
-   * @throws IOException if a remote or network exception occurs
-   */
-  HRegionLocation locateRegion(final byte[] regionName)
-  throws IOException;
-
-  /**
-   * Gets the locations of all regions in the specified table, <i>tableName</i>.
-   * @param tableName table to get regions of
-   * @return list of region locations for all regions of table
-   * @throws IOException if IO failure occurs
-   */
-  List<HRegionLocation> locateRegions(final TableName tableName) throws IOException;
-
-  /**
-   * Gets the locations of all regions in the specified table, <i>tableName</i>.
-   * @param tableName table to get regions of
-   * @param useCache Should we use the cache to retrieve the region information.
-   * @param offlined True if we are to include offlined regions, false and we'll leave out offlined
-   *          regions from returned list.
-   * @return list of region locations for all regions of table
-   * @throws IOException if IO failure occurs
-   */
-  List<HRegionLocation> locateRegions(final TableName tableName,
-      final boolean useCache,
-      final boolean offlined) throws IOException;
-
-  /**
-   *
-   * @param tableName table to get regions of
-   * @param row the row
-   * @param useCache Should we use the cache to retrieve the region information.
-   * @param retry do we retry
-   * @return region locations for this row.
-   * @throws IOException if IO failure occurs
-   */
-  RegionLocations locateRegion(TableName tableName,
-                               byte[] row, boolean useCache, boolean retry) throws IOException;
-
- /**
-  *
-  * @param tableName table to get regions of
-  * @param row the row
-  * @param useCache Should we use the cache to retrieve the region information.
-  * @param retry do we retry
-  * @param replicaId the replicaId for the region
-  * @return region locations for this row.
-  * @throws IOException if IO failure occurs
-  */
-  RegionLocations locateRegion(TableName tableName, byte[] row, boolean useCache, boolean retry,
-     int replicaId) throws IOException;
-
-  /**
    * Returns a {@link MasterKeepAliveConnection} to the active master
    */
   MasterKeepAliveConnection getMaster() throws IOException;
@@ -253,23 +121,6 @@ public interface ClusterConnection extends Connection {
   ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException;
 
   /**
-   * Find region location hosting passed row
-   * @param tableName table name
-   * @param row Row to find.
-   * @param reload If true do not use cache, otherwise bypass.
-   * @return Location of row.
-   * @throws IOException if a remote or network exception occurs
-   */
-  HRegionLocation getRegionLocation(TableName tableName, byte[] row, boolean reload)
-      throws IOException;
-
-  /**
-   * Clear any caches that pertain to server name <code>sn</code>.
-   * @param sn A server name
-   */
-  void clearCaches(final ServerName sn);
-
-  /**
    * @return Nonce generator for this ClusterConnection; may be null if disabled in configuration.
    */
   NonceGenerator getNonceGenerator();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index ff2ff2f..f262020 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -611,9 +611,16 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     return true;
   }
 
-  @Override
-  public HRegionLocation getRegionLocation(final TableName tableName, final byte[] row,
-      boolean reload) throws IOException {
+  /**
+   * Find region location hosting passed row
+   * @param tableName table name
+   * @param row Row to find.
+   * @param reload If true do not use cache, otherwise bypass.
+   * @return Location of row.
+   * @throws IOException if a remote or network exception occurs
+   */
+  HRegionLocation getRegionLocation(final TableName tableName, final byte[] row, boolean reload)
+      throws IOException {
     return reload ? relocateRegion(tableName, row) : locateRegion(tableName, row);
   }
 
@@ -683,13 +690,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     }
   }
 
-  @Override
-  public HRegionLocation locateRegion(final byte[] regionName) throws IOException {
-    RegionLocations locations = locateRegion(RegionInfo.getTable(regionName),
-      RegionInfo.getStartKey(regionName), false, true);
-    return locations == null ? null : locations.getRegionLocation();
-  }
-
   private boolean isDeadServer(ServerName sn) {
     if (clusterStatusListener == null) {
       return false;
@@ -698,13 +698,26 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     }
   }
 
-  @Override
-  public List<HRegionLocation> locateRegions(TableName tableName) throws IOException {
+  /**
+   * Gets the locations of all regions in the specified table, <i>tableName</i>.
+   * @param tableName table to get regions of
+   * @return list of region locations for all regions of table
+   * @throws IOException if IO failure occurs
+   */
+  List<HRegionLocation> locateRegions(TableName tableName) throws IOException {
     return locateRegions(tableName, false, true);
   }
 
-  @Override
-  public List<HRegionLocation> locateRegions(TableName tableName, boolean useCache,
+  /**
+   * Gets the locations of all regions in the specified table, <i>tableName</i>.
+   * @param tableName table to get regions of
+   * @param useCache Should we use the cache to retrieve the region information.
+   * @param offlined True if we are to include offlined regions, false and we'll leave out offlined
+   *          regions from returned list.
+   * @return list of region locations for all regions of table
+   * @throws IOException if IO failure occurs
+   */
+  List<HRegionLocation> locateRegions(TableName tableName, boolean useCache,
       boolean offlined) throws IOException {
     List<RegionInfo> regions;
     if (TableName.isMetaTableName(tableName)) {
@@ -729,24 +742,44 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     return locations;
   }
 
-  @Override
-  public HRegionLocation locateRegion(final TableName tableName, final byte[] row)
-      throws IOException {
+  /**
+   * Find the location of the region of <i>tableName</i> that <i>row</i> lives in.
+   * @param tableName name of the table <i>row</i> is in
+   * @param row row key you're trying to find the region of
+   * @return HRegionLocation that describes where to find the region in question
+   * @throws IOException if a remote or network exception occurs
+   */
+  HRegionLocation locateRegion(final TableName tableName, final byte[] row) throws IOException {
     RegionLocations locations = locateRegion(tableName, row, true, true);
     return locations == null ? null : locations.getRegionLocation();
   }
 
-  @Override
-  public HRegionLocation relocateRegion(final TableName tableName, final byte[] row)
-      throws IOException {
+  /**
+   * Find the location of the region of <i>tableName</i> that <i>row</i> lives in, ignoring any
+   * value that might be in the cache.
+   * @param tableName name of the table <i>row</i> is in
+   * @param row row key you're trying to find the region of
+   * @return HRegionLocation that describes where to find the region in question
+   * @throws IOException if a remote or network exception occurs
+   */
+  HRegionLocation relocateRegion(final TableName tableName, final byte[] row) throws IOException {
     RegionLocations locations =
       relocateRegion(tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID);
     return locations == null ? null
       : locations.getRegionLocation(RegionReplicaUtil.DEFAULT_REPLICA_ID);
   }
 
-  @Override
-  public RegionLocations relocateRegion(final TableName tableName,
+  /**
+   * Find the location of the region of <i>tableName</i> that <i>row</i>
+   * lives in, ignoring any value that might be in the cache.
+   * @param tableName name of the table <i>row</i> is in
+   * @param row row key you're trying to find the region of
+   * @param replicaId the replicaId of the region
+   * @return RegionLocations that describe where to find the region in
+   *   question
+   * @throws IOException if a remote or network exception occurs
+   */
+  RegionLocations relocateRegion(final TableName tableName,
       final byte [] row, int replicaId) throws IOException{
     // Since this is an explicit request not to use any caching, finding
     // disabled tables should not be desirable.  This will ensure that an exception is thrown when
@@ -758,14 +791,30 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     return locateRegion(tableName, row, false, true, replicaId);
   }
 
-  @Override
-  public RegionLocations locateRegion(final TableName tableName, final byte[] row, boolean useCache,
+  /**
+   * @param tableName table to get regions of
+   * @param row the row
+   * @param useCache Should we use the cache to retrieve the region information.
+   * @param retry do we retry
+   * @return region locations for this row.
+   * @throws IOException if IO failure occurs
+   */
+  RegionLocations locateRegion(final TableName tableName, final byte[] row, boolean useCache,
       boolean retry) throws IOException {
     return locateRegion(tableName, row, useCache, retry, RegionReplicaUtil.DEFAULT_REPLICA_ID);
   }
 
-  @Override
-  public RegionLocations locateRegion(final TableName tableName, final byte[] row, boolean useCache,
+  /**
+  *
+  * @param tableName table to get regions of
+  * @param row the row
+  * @param useCache Should we use the cache to retrieve the region information.
+  * @param retry do we retry
+  * @param replicaId the replicaId for the region
+  * @return region locations for this row.
+  * @throws IOException if IO failure occurs
+  */
+  RegionLocations locateRegion(final TableName tableName, final byte[] row, boolean useCache,
       boolean retry, int replicaId) throws IOException {
     checkClosed();
     if (tableName == null || tableName.getName().length == 0) {
@@ -969,8 +1018,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
    * @param tableName The table name.
    * @param location the new location
    */
-  @Override
-  public void cacheLocation(final TableName tableName, final RegionLocations location) {
+  void cacheLocation(final TableName tableName, final RegionLocations location) {
     metaCache.cacheLocation(tableName, location);
   }
 
@@ -984,15 +1032,15 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     return metaCache.getCachedLocation(tableName, row);
   }
 
-  public void clearRegionCache(final TableName tableName, byte[] row) {
+  void clearRegionCache(final TableName tableName, byte[] row) {
     metaCache.clearCache(tableName, row);
   }
 
-  /*
-   * Delete all cached entries of a table that maps to a specific location.
+  /**
+   * Clear any caches that pertain to server name <code>sn</code>.
+   * @param sn A server name
    */
-  @Override
-  public void clearCaches(final ServerName serverName) {
+  void clearCaches(final ServerName serverName) {
     metaCache.clearCache(serverName);
   }
 
@@ -1001,8 +1049,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     metaCache.clearCache();
   }
 
-  @Override
-  public void clearRegionCache(final TableName tableName) {
+  /**
+   * Allows flushing the region cache of all locations that pertain to <code>tableName</code>
+   * @param tableName Name of the table whose regions we are to remove from cache.
+   */
+  void clearRegionCache(final TableName tableName) {
     metaCache.clearCache(tableName);
   }
 
@@ -1853,8 +1904,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     cacheLocation(hri.getTable(), source, newHrl);
   }
 
-  @Override
-  public void deleteCachedRegionLocation(final HRegionLocation location) {
+  void deleteCachedRegionLocation(final HRegionLocation location) {
     metaCache.clearCache(location);
   }
 
@@ -1866,8 +1916,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
    *   or wrapped or both RegionMovedException
    * @param source server that is the source of the location update.
    */
-  @Override
-  public void updateCachedLocations(final TableName tableName, byte[] regionName, byte[] rowkey,
+  void updateCachedLocations(final TableName tableName, byte[] regionName, byte[] rowkey,
     final Object exception, final ServerName source) {
     if (rowkey == null || tableName == null) {
       LOG.warn("Coding error, see method javadoc. row=" + (rowkey == null ? "null" : rowkey) +
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
index bb265a4..d881fe0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
@@ -28,6 +27,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
@@ -42,7 +42,7 @@ public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionR
   private final boolean writeFlushWalMarker;
   private boolean reload;
 
-  public FlushRegionCallable(ClusterConnection connection,
+  public FlushRegionCallable(ConnectionImplementation connection,
       RpcControllerFactory rpcControllerFactory, TableName tableName, byte[] regionName,
       byte[] regionStartKey, boolean writeFlushWalMarker) {
     super(connection, rpcControllerFactory, tableName, regionStartKey);
@@ -50,7 +50,7 @@ public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionR
     this.writeFlushWalMarker = writeFlushWalMarker;
   }
 
-  public FlushRegionCallable(ClusterConnection connection,
+  public FlushRegionCallable(ConnectionImplementation connection,
       RpcControllerFactory rpcControllerFactory, RegionInfo regionInfo,
       boolean writeFlushWalMarker) {
     this(connection, rpcControllerFactory, regionInfo.getTable(), regionInfo.getRegionName(),
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 6a38ead..d16d630 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcController;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -45,7 +44,6 @@ import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CacheEvictionStats;
@@ -249,7 +247,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
 public class HBaseAdmin implements Admin {
   private static final Logger LOG = LoggerFactory.getLogger(HBaseAdmin.class);
 
-  private ClusterConnection connection;
+  private ConnectionImplementation connection;
 
   private final Configuration conf;
   private final long pause;
@@ -270,7 +268,7 @@ public class HBaseAdmin implements Admin {
     return operationTimeout;
   }
 
-  HBaseAdmin(ClusterConnection connection) throws IOException {
+  HBaseAdmin(ConnectionImplementation connection) throws IOException {
     this.conf = connection.getConfiguration();
     this.connection = connection;
 
@@ -756,7 +754,9 @@ public class HBaseAdmin implements Admin {
     protected Void postOperationResult(final Void result, final long deadlineTs)
         throws IOException, TimeoutException {
       // Delete cached information to prevent clients from using old locations
-      ((ClusterConnection) getAdmin().getConnection()).clearRegionCache(getTableName());
+      try (RegionLocator locator = getAdmin().getConnection().getRegionLocator(getTableName())) {
+        locator.clearRegionLocationCache();
+      }
       return super.postOperationResult(result, deadlineTs);
     }
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 9b3afd9..cc796c8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -104,7 +104,7 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
 public class HTable implements Table {
   private static final Logger LOG = LoggerFactory.getLogger(HTable.class);
   private static final Consistency DEFAULT_CONSISTENCY = Consistency.STRONG;
-  private final ClusterConnection connection;
+  private final ConnectionImplementation connection;
   private final TableName tableName;
   private final Configuration configuration;
   private final ConnectionConfiguration connConfiguration;
@@ -936,7 +936,7 @@ public class HTable implements Table {
    * This is a power user function: avoid unless you know the ramifications.
    */
   public void clearRegionCache() {
-    this.connection.clearRegionCache();
+    this.connection.clearRegionLocationCache();
   }
 
   @Override
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
index ece91bd..33dfefa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
@@ -46,25 +46,25 @@ public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<
   protected final RpcControllerFactory rpcControllerFactory;
   private HBaseRpcController controller = null;
 
-  protected final ClusterConnection connection;
+  protected final ConnectionImplementation connection;
   protected HRegionLocation location;
   protected final TableName tableName;
   protected final byte[] row;
   protected final int replicaId;
 
-  public RegionAdminServiceCallable(ClusterConnection connection,
+  public RegionAdminServiceCallable(ConnectionImplementation connection,
       RpcControllerFactory rpcControllerFactory, TableName tableName, byte[] row) {
     this(connection, rpcControllerFactory, null, tableName, row);
   }
 
-  public RegionAdminServiceCallable(ClusterConnection connection,
+  public RegionAdminServiceCallable(ConnectionImplementation connection,
       RpcControllerFactory rpcControllerFactory, HRegionLocation location,
       TableName tableName, byte[] row) {
     this(connection, rpcControllerFactory, location,
       tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID);
   }
 
-  public RegionAdminServiceCallable(ClusterConnection connection,
+  public RegionAdminServiceCallable(ConnectionImplementation connection,
       RpcControllerFactory rpcControllerFactory, HRegionLocation location,
       TableName tableName, byte[] row, int replicaId) {
     this.connection = connection;
@@ -138,8 +138,8 @@ public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<
     return ConnectionUtils.getPauseTime(pause, tries);
   }
 
-  public static RegionLocations getRegionLocations(
-      ClusterConnection connection, TableName tableName, byte[] row,
+  private static RegionLocations getRegionLocations(
+      ConnectionImplementation connection, TableName tableName, byte[] row,
       boolean useCache, int replicaId)
       throws RetriesExhaustedException, DoNotRetryIOException, InterruptedIOException {
     RegionLocations rl;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
index f709c44..264304e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
@@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 // Public but should be package private only it is used by MetaTableAccessor. FIX!!
 @InterfaceAudience.Private
 public abstract class RegionServerCallable<T, S> implements RetryingCallable<T> {
-  private final Connection connection;
+  private final ConnectionImplementation connection;
   private final TableName tableName;
   private final byte[] row;
   /**
@@ -75,12 +75,12 @@ public abstract class RegionServerCallable<T, S> implements RetryingCallable<T>
    * @param tableName Table name to which <code>row</code> belongs.
    * @param row The row we want in <code>tableName</code>.
    */
-  public RegionServerCallable(Connection connection, TableName tableName, byte [] row,
+  public RegionServerCallable(ConnectionImplementation connection, TableName tableName, byte [] row,
       RpcController rpcController) {
     this(connection, tableName, row, rpcController, HConstants.NORMAL_QOS);
   }
 
-  public RegionServerCallable(Connection connection, TableName tableName, byte [] row,
+  public RegionServerCallable(ConnectionImplementation connection, TableName tableName, byte [] row,
       RpcController rpcController, int priority) {
     super();
     this.connection = connection;
@@ -162,8 +162,8 @@ public abstract class RegionServerCallable<T, S> implements RetryingCallable<T>
   /**
    * @return {@link ClusterConnection} instance used by this Callable.
    */
-  protected ClusterConnection getConnection() {
-    return (ClusterConnection) this.connection;
+  protected ConnectionImplementation getConnection() {
+    return this.connection;
   }
 
   protected HRegionLocation getLocation() {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java
index 53b9641..34c24c0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java
@@ -46,11 +46,11 @@ public class ReversedClientScanner extends ClientScanner {
    * @throws IOException
    */
   public ReversedClientScanner(Configuration conf, Scan scan, TableName tableName,
-      ClusterConnection connection, RpcRetryingCallerFactory rpcFactory,
+      ConnectionImplementation connection, RpcRetryingCallerFactory rpcFactory,
       RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout)
       throws IOException {
     super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool,
-        primaryOperationTimeout);
+      primaryOperationTimeout);
   }
 
   @Override
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
index 4a31cff..b2d0600 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hbase.client;
 
 
+import static org.apache.hadoop.hbase.HConstants.PRIORITY_UNSET;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.Collections;
@@ -27,24 +29,22 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
-import static org.apache.hadoop.hbase.HConstants.PRIORITY_UNSET;
 
 /**
  * Caller that goes to replica if the primary region does no answer within a configurable
@@ -58,7 +58,7 @@ public class RpcRetryingCallerWithReadReplicas {
       LoggerFactory.getLogger(RpcRetryingCallerWithReadReplicas.class);
 
   protected final ExecutorService pool;
-  protected final ClusterConnection cConnection;
+  protected final ConnectionImplementation cConnection;
   protected final Configuration conf;
   protected final Get get;
   protected final TableName tableName;
@@ -71,7 +71,7 @@ public class RpcRetryingCallerWithReadReplicas {
 
   public RpcRetryingCallerWithReadReplicas(
       RpcControllerFactory rpcControllerFactory, TableName tableName,
-      ClusterConnection cConnection, final Get get,
+      ConnectionImplementation cConnection, final Get get,
       ExecutorService pool, int retries, int operationTimeout, int rpcTimeout,
       int timeBeforeReplicas) {
     this.rpcControllerFactory = rpcControllerFactory;
@@ -185,19 +185,14 @@ public class RpcRetryingCallerWithReadReplicas {
       } else {
         // We cannot get the primary replica location, it is possible that the region
         // server hosting meta is down, it needs to proceed to try cached replicas.
-        if (cConnection instanceof ConnectionImplementation) {
-          rl = ((ConnectionImplementation)cConnection).getCachedLocation(tableName, get.getRow());
-          if (rl == null) {
-            // No cached locations
-            throw e;
-          }
-
-          // Primary replica location is not known, skip primary replica
-          skipPrimary = true;
-        } else {
-          // For completeness
+        rl = cConnection.getCachedLocation(tableName, get.getRow());
+        if (rl == null) {
+          // No cached locations
           throw e;
         }
+
+        // Primary replica location is not known, skip primary replica
+        skipPrimary = true;
       }
     }
 
@@ -316,9 +311,8 @@ public class RpcRetryingCallerWithReadReplicas {
   }
 
   static RegionLocations getRegionLocations(boolean useCache, int replicaId,
-                 ClusterConnection cConnection, TableName tableName, byte[] row)
+      ConnectionImplementation cConnection, TableName tableName, byte[] row)
       throws RetriesExhaustedException, DoNotRetryIOException, InterruptedIOException {
-
     RegionLocations rl;
     try {
       if (useCache) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
index bcb81f7..27e5f87 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hbase.client;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.HashSet;
@@ -31,17 +29,18 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults;
-import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
  * This class has the logic for handling scanners for regions with and without replicas.
@@ -60,7 +59,7 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
   private static final Logger LOG = LoggerFactory.getLogger(ScannerCallableWithReplicas.class);
   volatile ScannerCallable currentScannerCallable;
   AtomicBoolean replicaSwitched = new AtomicBoolean(false);
-  final ClusterConnection cConnection;
+  final ConnectionImplementation cConnection;
   protected final ExecutorService pool;
   protected final int timeBeforeReplicas;
   private final Scan scan;
@@ -74,7 +73,7 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
   private boolean someRPCcancelled = false; //required for testing purposes only
   private int regionReplication = 0;
 
-  public ScannerCallableWithReplicas(TableName tableName, ClusterConnection cConnection,
+  public ScannerCallableWithReplicas(TableName tableName, ConnectionImplementation cConnection,
       ScannerCallable baseCallable, ExecutorService pool, int timeBeforeReplicas, Scan scan,
       int retries, int scannerTimeout, int caching, Configuration conf,
       RpcRetryingCaller<Result []> caller) {
@@ -151,19 +150,13 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
       RegionLocations rl = null;
       try {
         rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(true,
-            RegionReplicaUtil.DEFAULT_REPLICA_ID, cConnection, tableName,
-            currentScannerCallable.getRow());
+          RegionReplicaUtil.DEFAULT_REPLICA_ID, cConnection, tableName,
+          currentScannerCallable.getRow());
       } catch (RetriesExhaustedException | DoNotRetryIOException e) {
         // We cannot get the primary replica region location, it is possible that the region server
         // hosting meta table is down, it needs to proceed to try cached replicas directly.
-        if (cConnection instanceof ConnectionImplementation) {
-          rl = ((ConnectionImplementation) cConnection)
-              .getCachedLocation(tableName, currentScannerCallable.getRow());
-          if (rl == null) {
-            throw e;
-          }
-        } else {
-          // For completeness
+        rl = cConnection.getCachedLocation(tableName, currentScannerCallable.getRow());
+        if (rl == null) {
           throw e;
         }
       }
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index 71b21ac..70d10ca 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -175,17 +175,17 @@ public class TestAsyncProcess {
       return r;
     }
 
-    public MyAsyncProcess(ClusterConnection hc, Configuration conf) {
-      super(hc, conf,
-          new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf));
+    public MyAsyncProcess(ConnectionImplementation hc, Configuration conf) {
+      super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf));
       service = Executors.newFixedThreadPool(5);
       this.conf = conf;
     }
 
-    public MyAsyncProcess(ClusterConnection hc, Configuration conf, AtomicInteger nbThreads) {
+    public MyAsyncProcess(ConnectionImplementation hc, Configuration conf,
+        AtomicInteger nbThreads) {
       super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf));
-      service = new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
-          new SynchronousQueue<>(), new CountingThreadFactory(nbThreads));
+      service = new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, new SynchronousQueue<>(),
+        new CountingThreadFactory(nbThreads));
     }
 
     public <CResult> AsyncRequestFuture submit(ExecutorService pool, TableName tableName,
@@ -326,7 +326,8 @@ public class TestAsyncProcess {
 
     private final IOException ioe;
 
-    public AsyncProcessWithFailure(ClusterConnection hc, Configuration conf, IOException ioe) {
+    public AsyncProcessWithFailure(ConnectionImplementation hc, Configuration conf,
+        IOException ioe) {
       super(hc, conf);
       this.ioe = ioe;
       serverTrackerTimeout = 1L;
@@ -376,7 +377,7 @@ public class TestAsyncProcess {
       customPrimarySleepMs.put(server, primaryMs);
     }
 
-    public MyAsyncProcessWithReplicas(ClusterConnection hc, Configuration conf) {
+    public MyAsyncProcessWithReplicas(ConnectionImplementation hc, Configuration conf) {
       super(hc, conf);
     }
 
@@ -622,7 +623,7 @@ public class TestAsyncProcess {
   }
 
   private void doSubmitRequest(long maxHeapSizePerRequest, long putsHeapSize) throws Exception {
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     final String defaultClazz =
         conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
     final long defaultHeapSizePerRequest = conn.getConfiguration().getLong(
@@ -718,7 +719,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testSubmit() throws Exception {
-    ClusterConnection hc = createHConnection();
+    ConnectionImplementation hc = createConnectionImpl();
     MyAsyncProcess ap = new MyAsyncProcess(hc, CONF);
 
     List<Put> puts = new ArrayList<>(1);
@@ -730,7 +731,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testSubmitWithCB() throws Exception {
-    ClusterConnection hc = createHConnection();
+    ConnectionImplementation hc = createConnectionImpl();
     final AtomicInteger updateCalled = new AtomicInteger(0);
     Batch.Callback<Object> cb = new Batch.Callback<Object>() {
       @Override
@@ -751,7 +752,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testSubmitBusyRegion() throws Exception {
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     final String defaultClazz =
         conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
     conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
@@ -779,7 +780,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testSubmitBusyRegionServer() throws Exception {
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     final String defaultClazz =
         conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
@@ -810,7 +811,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testFail() throws Exception {
-    MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF);
+    MyAsyncProcess ap = new MyAsyncProcess(createConnectionImpl(), CONF);
 
     List<Put> puts = new ArrayList<>(1);
     Put p = createPut(1, false);
@@ -836,7 +837,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testSubmitTrue() throws IOException {
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     final MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     final String defaultClazz =
         conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
@@ -885,7 +886,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testFailAndSuccess() throws Exception {
-    MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF);
+    MyAsyncProcess ap = new MyAsyncProcess(createConnectionImpl(), CONF);
 
     List<Put> puts = new ArrayList<>(3);
     puts.add(createPut(1, false));
@@ -912,7 +913,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testFlush() throws Exception {
-    MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF);
+    MyAsyncProcess ap = new MyAsyncProcess(createConnectionImpl(), CONF);
 
     List<Put> puts = new ArrayList<>(3);
     puts.add(createPut(1, false));
@@ -929,7 +930,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testTaskCountWithoutClientBackoffPolicy() throws IOException, InterruptedException {
-    ClusterConnection hc = createHConnection();
+    ConnectionImplementation hc = createConnectionImpl();
     MyAsyncProcess ap = new MyAsyncProcess(hc, CONF);
     testTaskCount(ap);
   }
@@ -939,7 +940,7 @@ public class TestAsyncProcess {
     Configuration copyConf = new Configuration(CONF);
     copyConf.setBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE, true);
     MyClientBackoffPolicy bp = new MyClientBackoffPolicy();
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     Mockito.when(conn.getConfiguration()).thenReturn(copyConf);
     Mockito.when(conn.getStatisticsTracker()).thenReturn(ServerStatisticTracker.create(copyConf));
     Mockito.when(conn.getBackoffPolicy()).thenReturn(bp);
@@ -979,7 +980,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testMaxTask() throws Exception {
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     final String defaultClazz =
         conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
     conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
@@ -1038,8 +1039,8 @@ public class TestAsyncProcess {
     }
   }
 
-  private ClusterConnection createHConnection() throws IOException {
-    ClusterConnection hc = createHConnectionCommon();
+  private ConnectionImplementation createConnectionImpl() throws IOException {
+    ConnectionImplementation hc = createConnectionImplCommon();
     setMockLocation(hc, DUMMY_BYTES_1, new RegionLocations(loc1));
     setMockLocation(hc, DUMMY_BYTES_2, new RegionLocations(loc2));
     setMockLocation(hc, DUMMY_BYTES_3, new RegionLocations(loc3));
@@ -1049,8 +1050,8 @@ public class TestAsyncProcess {
     return hc;
   }
 
-  private ClusterConnection createHConnectionWithReplicas() throws IOException {
-    ClusterConnection hc = createHConnectionCommon();
+  private ConnectionImplementation createConnectionImplWithReplicas() throws IOException {
+    ConnectionImplementation hc = createConnectionImplCommon();
     setMockLocation(hc, DUMMY_BYTES_1, hrls1);
     setMockLocation(hc, DUMMY_BYTES_2, hrls2);
     setMockLocation(hc, DUMMY_BYTES_3, hrls3);
@@ -1069,16 +1070,16 @@ public class TestAsyncProcess {
     return hc;
   }
 
-  private static void setMockLocation(ClusterConnection hc, byte[] row,
+  private static void setMockLocation(ConnectionImplementation hc, byte[] row,
       RegionLocations result) throws IOException {
-    Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row),
-        Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyInt())).thenReturn(result);
-    Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row),
-        Mockito.anyBoolean(), Mockito.anyBoolean())).thenReturn(result);
+    Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row), Mockito.anyBoolean(),
+      Mockito.anyBoolean(), Mockito.anyInt())).thenReturn(result);
+    Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row), Mockito.anyBoolean(),
+      Mockito.anyBoolean())).thenReturn(result);
   }
 
-  private ClusterConnection createHConnectionCommon() {
-    ClusterConnection hc = Mockito.mock(ClusterConnection.class);
+  private ConnectionImplementation createConnectionImplCommon() {
+    ConnectionImplementation hc = Mockito.mock(ConnectionImplementation.class);
     NonceGenerator ng = Mockito.mock(NonceGenerator.class);
     Mockito.when(ng.getNonceGroup()).thenReturn(HConstants.NO_NONCE);
     Mockito.when(hc.getNonceGenerator()).thenReturn(ng);
@@ -1089,7 +1090,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testHTablePutSuccess() throws Exception {
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
     BufferedMutatorImpl ht = new BufferedMutatorImpl(conn, bufferParam, ap);
@@ -1106,7 +1107,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testSettingWriteBufferPeriodicFlushParameters() throws Exception {
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
 
     checkPeriodicFlushParameters(conn, ap,
@@ -1152,7 +1153,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testWriteBufferPeriodicFlushTimeoutMs() throws Exception {
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
 
@@ -1219,7 +1220,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testBufferedMutatorImplWithSharedPool() throws Exception {
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
     BufferedMutator ht = new BufferedMutatorImpl(conn, bufferParam, ap);
@@ -1230,7 +1231,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testFailedPutAndNewPut() throws Exception {
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE)
             .writeBufferSize(0);
@@ -1275,7 +1276,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testBatch() throws IOException, InterruptedException {
-    ClusterConnection conn = new MyConnectionImpl(CONF);
+    ConnectionImplementation conn = new MyConnectionImpl(CONF);
     HTable ht = (HTable) conn.getTable(DUMMY_TABLE);
     ht.multiAp = new MyAsyncProcess(conn, CONF);
 
@@ -1306,7 +1307,7 @@ public class TestAsyncProcess {
   @Test
   public void testErrorsServers() throws IOException {
     Configuration configuration = new Configuration(CONF);
-    ClusterConnection conn = new MyConnectionImpl(configuration);
+    ConnectionImplementation conn = new MyConnectionImpl(configuration);
     MyAsyncProcess ap = new MyAsyncProcess(conn, configuration);
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
     BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap);
@@ -1337,7 +1338,7 @@ public class TestAsyncProcess {
     Configuration copyConf = new Configuration(CONF);
     copyConf.setLong(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, readTimeout);
     copyConf.setLong(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, writeTimeout);
-    ClusterConnection conn = new MyConnectionImpl(copyConf);
+    ConnectionImplementation conn = new MyConnectionImpl(copyConf);
     MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf);
     try (HTable ht = (HTable) conn.getTable(DUMMY_TABLE)) {
       ht.multiAp = ap;
@@ -1370,7 +1371,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testErrors() throws IOException {
-    ClusterConnection conn = new MyConnectionImpl(CONF);
+    ConnectionImplementation conn = new MyConnectionImpl(CONF);
     AsyncProcessWithFailure ap = new AsyncProcessWithFailure(conn, CONF, new IOException("test"));
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
     BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap);
@@ -1394,7 +1395,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testCallQueueTooLarge() throws IOException {
-    ClusterConnection conn = new MyConnectionImpl(CONF);
+    ConnectionImplementation conn = new MyConnectionImpl(CONF);
     AsyncProcessWithFailure ap =
         new AsyncProcessWithFailure(conn, CONF, new CallQueueTooBigException());
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
@@ -1609,7 +1610,7 @@ public class TestAsyncProcess {
     // TODO: this is kind of timing dependent... perhaps it should detect from createCaller
     //       that the replica call has happened and that way control the ordering.
     Configuration conf = new Configuration();
-    ClusterConnection conn = createHConnectionWithReplicas();
+    ConnectionImplementation conn = createConnectionImplWithReplicas();
     conf.setInt(AsyncProcess.PRIMARY_CALL_TIMEOUT_KEY, replicaAfterMs * 1000);
     if (retries >= 0) {
       conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
@@ -1707,16 +1708,15 @@ public class TestAsyncProcess {
   }
 
   static class AsyncProcessForThrowableCheck extends AsyncProcess {
-    public AsyncProcessForThrowableCheck(ClusterConnection hc, Configuration conf) {
-      super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(
-          conf));
+    public AsyncProcessForThrowableCheck(ConnectionImplementation hc, Configuration conf) {
+      super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf));
     }
   }
 
   @Test
   public void testUncheckedException() throws Exception {
     // Test the case pool.submit throws unchecked exception
-    ClusterConnection hc = createHConnection();
+    ConnectionImplementation hc = createConnectionImpl();
     MyThreadPoolExecutor myPool =
         new MyThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
             new LinkedBlockingQueue<>(200));
@@ -1748,7 +1748,7 @@ public class TestAsyncProcess {
     final int retries = 1;
     myConf.setLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, specialPause);
     myConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
-    ClusterConnection conn = new MyConnectionImpl(myConf);
+    ConnectionImplementation conn = new MyConnectionImpl(myConf);
     AsyncProcessWithFailure ap =
         new AsyncProcessWithFailure(conn, myConf, new CallQueueTooBigException());
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
@@ -1807,7 +1807,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testRetryWithExceptionClearsMetaCache() throws Exception {
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     Configuration myConf = conn.getConfiguration();
     myConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0);
 
@@ -1840,7 +1840,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testQueueRowAccess() throws Exception {
-    ClusterConnection conn = createHConnection();
+    ConnectionImplementation conn = createConnectionImpl();
     BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, null, null,
       new BufferedMutatorParams(DUMMY_TABLE).writeBufferSize(100000));
     Put p0 = new Put(DUMMY_BYTES_1).addColumn(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1);
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
index ffc4e51..2c24aaa 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
@@ -175,8 +175,8 @@ public class TestAsyncProcessWithRegionException {
     }
   }
 
-  private static ClusterConnection createHConnection() throws IOException {
-    ClusterConnection hc = Mockito.mock(ClusterConnection.class);
+  private static ConnectionImplementation createHConnection() throws IOException {
+    ConnectionImplementation hc = Mockito.mock(ConnectionImplementation.class);
     NonceGenerator ng = Mockito.mock(NonceGenerator.class);
     Mockito.when(ng.getNonceGroup()).thenReturn(HConstants.NO_NONCE);
     Mockito.when(hc.getNonceGenerator()).thenReturn(ng);
@@ -190,8 +190,8 @@ public class TestAsyncProcessWithRegionException {
     return hc;
   }
 
-  private static void setMockLocation(ClusterConnection hc, byte[] row, RegionLocations result)
-    throws IOException {
+  private static void setMockLocation(ConnectionImplementation hc, byte[] row,
+      RegionLocations result) throws IOException {
     Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row), Mockito.anyBoolean(),
       Mockito.anyBoolean(), Mockito.anyInt())).thenReturn(result);
     Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row), Mockito.anyBoolean(),
@@ -201,7 +201,7 @@ public class TestAsyncProcessWithRegionException {
   private static class MyAsyncProcess extends AsyncProcess {
     private final ExecutorService service = Executors.newFixedThreadPool(5);
 
-    MyAsyncProcess(ClusterConnection hc, Configuration conf) {
+    MyAsyncProcess(ConnectionImplementation hc, Configuration conf) {
       super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf));
     }
 
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java
index f8e1295..f0375e2 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java
@@ -48,7 +48,7 @@ public class TestBufferedMutator {
    * Just to prove that I can insert a BM other than default.
    */
   public static class MyBufferedMutator extends BufferedMutatorImpl {
-    MyBufferedMutator(ClusterConnection conn, RpcRetryingCallerFactory rpcCallerFactory,
+    MyBufferedMutator(ConnectionImplementation conn, RpcRetryingCallerFactory rpcCallerFactory,
         RpcControllerFactory rpcFactory, BufferedMutatorParams params) {
       super(conn, rpcCallerFactory, rpcFactory, params);
     }
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
index 48ca751..9f1f6f3 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
@@ -71,7 +71,7 @@ public class TestClientScanner {
   ExecutorService pool;
   Configuration conf;
 
-  ClusterConnection clusterConn;
+  ConnectionImplementation clusterConn;
   RpcRetryingCallerFactory rpcFactory;
   RpcControllerFactory controllerFactory;
 
@@ -80,7 +80,7 @@ public class TestClientScanner {
 
   @Before
   public void setup() throws IOException {
-    clusterConn = Mockito.mock(ClusterConnection.class);
+    clusterConn = Mockito.mock(ConnectionImplementation.class);
     rpcFactory = Mockito.mock(RpcRetryingCallerFactory.class);
     controllerFactory = Mockito.mock(RpcControllerFactory.class);
     pool = Executors.newSingleThreadExecutor();
@@ -103,11 +103,11 @@ public class TestClientScanner {
     private boolean initialized = false;
 
     public MockClientScanner(final Configuration conf, final Scan scan, final TableName tableName,
-        ClusterConnection connection, RpcRetryingCallerFactory rpcFactory,
+        ConnectionImplementation connection, RpcRetryingCallerFactory rpcFactory,
         RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout)
         throws IOException {
       super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool,
-          primaryOperationTimeout);
+        primaryOperationTimeout);
     }
 
     @Override
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java
index 1b554f7..7eb2b94 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
@@ -46,7 +45,7 @@ public class TestReversedScannerCallable {
       HBaseClassTestRule.forClass(TestReversedScannerCallable.class);
 
   @Mock
-  private ClusterConnection connection;
+  private ConnectionImplementation connection;
   @Mock
   private Scan scan;
   @Mock
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java
index b5e99d2..b9cb167 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java
@@ -31,13 +31,12 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.IntegrationTestIngest;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
-import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
@@ -52,6 +51,7 @@ import org.junit.Assert;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
@@ -350,10 +350,11 @@ public class IntegrationTestTimeBoundedRequestsWithRegionReplicas extends Integr
           numReadFailures.addAndGet(1); // fail the test
           for (Result r : results) {
             LOG.error("FAILED FOR " + r);
-            RegionLocations rl = ((ClusterConnection)connection).
-                locateRegion(tableName, r.getRow(), true, true);
-            HRegionLocation locations[] = rl.getRegionLocations();
-            for (HRegionLocation h : locations) {
+            List<HRegionLocation> locs;
+            try (RegionLocator locator = connection.getRegionLocator(tableName)) {
+              locs = locator.getRegionLocations(r.getRow());
+            }
+            for (HRegionLocation h : locs) {
               LOG.error("LOCATION " + h);
             }
           }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 9e5f9e8..34c846d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -99,6 +99,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -1948,35 +1949,34 @@ public class HBaseFsck extends Configured implements Closeable {
    * Record the location of the hbase:meta region as found in ZooKeeper.
    */
   private boolean recordMetaRegion() throws IOException {
-    RegionLocations rl = connection.locateRegion(TableName.META_TABLE_NAME,
-        HConstants.EMPTY_START_ROW, false, false);
-    if (rl == null) {
-      errors.reportError(ERROR_CODE.NULL_META_REGION,
-          "META region was not found in ZooKeeper");
+    List<HRegionLocation> locs;
+    try (RegionLocator locator = connection.getRegionLocator(TableName.META_TABLE_NAME)) {
+      locs = locator.getRegionLocations(HConstants.EMPTY_START_ROW, true);
+    }
+    if (locs == null || locs.isEmpty()) {
+      errors.reportError(ERROR_CODE.NULL_META_REGION, "META region was not found in ZooKeeper");
       return false;
     }
-    for (HRegionLocation metaLocation : rl.getRegionLocations()) {
+    for (HRegionLocation metaLocation : locs) {
       // Check if Meta region is valid and existing
-      if (metaLocation == null ) {
-        errors.reportError(ERROR_CODE.NULL_META_REGION,
-            "META region location is null");
+      if (metaLocation == null) {
+        errors.reportError(ERROR_CODE.NULL_META_REGION, "META region location is null");
         return false;
       }
-      if (metaLocation.getRegionInfo() == null) {
-        errors.reportError(ERROR_CODE.NULL_META_REGION,
-            "META location regionInfo is null");
+      if (metaLocation.getRegion() == null) {
+        errors.reportError(ERROR_CODE.NULL_META_REGION, "META location regionInfo is null");
         return false;
       }
       if (metaLocation.getHostname() == null) {
-        errors.reportError(ERROR_CODE.NULL_META_REGION,
-            "META location hostName is null");
+        errors.reportError(ERROR_CODE.NULL_META_REGION, "META location hostName is null");
         return false;
       }
       ServerName sn = metaLocation.getServerName();
-      MetaEntry m = new MetaEntry(metaLocation.getRegionInfo(), sn, EnvironmentEdgeManager.currentTime());
-      HbckInfo hbckInfo = regionInfoMap.get(metaLocation.getRegionInfo().getEncodedName());
+      MetaEntry m =
+        new MetaEntry(metaLocation.getRegion(), sn, EnvironmentEdgeManager.currentTime());
+      HbckInfo hbckInfo = regionInfoMap.get(metaLocation.getRegion().getEncodedName());
       if (hbckInfo == null) {
-        regionInfoMap.put(metaLocation.getRegionInfo().getEncodedName(), new HbckInfo(m));
+        regionInfoMap.put(metaLocation.getRegion().getEncodedName(), new HbckInfo(m));
       } else {
         hbckInfo.metaEntry = m;
       }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
index 1b58634..a779d36 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
@@ -44,22 +44,21 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+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.NoServerForRegionException;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
@@ -707,7 +706,7 @@ public class RegionSplitter {
     Path tableDir = tableDirAndSplitFile.getFirst();
     FileSystem fs = tableDir.getFileSystem(connection.getConfiguration());
     // Clear the cache to forcibly refresh region information
-    ((ClusterConnection)connection).clearRegionCache();
+    connection.clearRegionLocationCache();
     TableDescriptor htd = null;
     try (Table table = connection.getTable(tableName)) {
       htd = table.getDescriptor();
@@ -768,7 +767,7 @@ public class RegionSplitter {
         } catch (NoServerForRegionException nsfre) {
           LOG.debug("No Server Exception thrown for: " + splitAlgo.rowToStr(start));
           physicalSplitting.add(region);
-          ((ClusterConnection)connection).clearRegionCache();
+          connection.clearRegionLocationCache();
         }
       }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
index a8beab6..2a5a395 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
@@ -20,44 +20,43 @@ package org.apache.hadoop.hbase.client;
 import java.io.IOException;
 import java.util.Optional;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.util.Threads;
 import org.mockito.Mockito;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+
 /**
- * {@link ClusterConnection} testing utility.
+ * {@link ConnectionImplementation} testing utility.
  */
 public class HConnectionTestingUtility {
   /*
-   * Not part of {@link HBaseTestingUtility} because this class is not
-   * in same package as {@link ClusterConnection}.  Would have to reveal ugly
-   * {@link ConnectionImplementation} innards to HBaseTestingUtility to give it access.
+   * Not part of {@link HBaseTestingUtility} because this class is not in same package as {@link
+   * ConnectionImplementation}. Would have to reveal ugly {@link ConnectionImplementation} innards
+   * to HBaseTestingUtility to give it access.
    */
   /**
-   * Get a Mocked {@link ClusterConnection} that goes with the passed <code>conf</code>
-   * configuration instance.  Minimally the mock will return
-   * &lt;code>conf&lt;/conf> when {@link ClusterConnection#getConfiguration()} is invoked.
-   * Be sure to shutdown the connection when done by calling
-   * {@link Connection#close()} else it will stick around; this is probably not what you want.
+   * Get a Mocked {@link ConnectionImplementation} that goes with the passed <code>conf</code>
+   * configuration instance. Minimally the mock will return &lt;code>conf&lt;/conf> when
+   * {@link ConnectionImplementation#getConfiguration()} is invoked. Be sure to shutdown the
+   * connection when done by calling {@link Connection#close()} else it will stick around; this is
+   * probably not what you want.
    * @param conf configuration
-   * @return ClusterConnection object for <code>conf</code>
+   * @return ConnectionImplementation object for <code>conf</code>
    * @throws ZooKeeperConnectionException
    */
-  public static ClusterConnection getMockedConnection(final Configuration conf)
+  public static ConnectionImplementation getMockedConnection(final Configuration conf)
   throws ZooKeeperConnectionException {
     ConnectionImplementation connection = Mockito.mock(ConnectionImplementation.class);
     Mockito.when(connection.getConfiguration()).thenReturn(conf);
@@ -70,37 +69,30 @@ public class HConnectionTestingUtility {
   }
 
   /**
-   * Calls {@link #getMockedConnection(Configuration)} and then mocks a few
-   * more of the popular {@link ClusterConnection} methods so they do 'normal'
-   * operation (see return doc below for list). Be sure to shutdown the
-   * connection when done by calling {@link Connection#close()} else it will stick around;
-   * this is probably not what you want.
-   *
+   * Calls {@link #getMockedConnection(Configuration)} and then mocks a few more of the popular
+   * {@link ConnectionImplementation} methods so they do 'normal' operation (see return doc below
+   * for list). Be sure to shutdown the connection when done by calling {@link Connection#close()}
+   * else it will stick around; this is probably not what you want.
    * @param conf Configuration to use
-   * @param admin An AdminProtocol; can be null but is usually
-   * itself a mock.
-   * @param client A ClientProtocol; can be null but is usually
-   * itself a mock.
-   * @param sn ServerName to include in the region location returned by this
-   * <code>connection</code>
-   * @param hri RegionInfo to include in the location returned when
-   * getRegionLocator is called on the mocked connection
+   * @param admin An AdminProtocol; can be null but is usually itself a mock.
+   * @param client A ClientProtocol; can be null but is usually itself a mock.
+   * @param sn ServerName to include in the region location returned by this <code>connection</code>
+   * @param hri RegionInfo to include in the location returned when getRegionLocator is called on
+   *          the mocked connection
    * @return Mock up a connection that returns a {@link Configuration} when
-   * {@link ClusterConnection#getConfiguration()} is called, a 'location' when
-   * {@link ClusterConnection#getRegionLocation(org.apache.hadoop.hbase.TableName, byte[], boolean)}
-   * is called,
-   * and that returns the passed {@link AdminProtos.AdminService.BlockingInterface} instance when
-   * {@link ClusterConnection#getAdmin(ServerName)} is called, returns the passed
-   * {@link ClientProtos.ClientService.BlockingInterface} instance when
-   * {@link ClusterConnection#getClient(ServerName)} is called (Be sure to call
-   * {@link Connection#close()} when done with this mocked Connection.
-   * @throws IOException
+   *         {@link ConnectionImplementation#getConfiguration()} is called, a 'location' when
+   *         {@link ConnectionImplementation#getRegionLocation(TableName,byte[], boolean)}
+   *         is called, and that returns the passed
+   *         {@link AdminProtos.AdminService.BlockingInterface} instance when
+   *         {@link ConnectionImplementation#getAdmin(ServerName)} is called, returns the passed
+   *         {@link ClientProtos.ClientService.BlockingInterface} instance when
+   *         {@link ConnectionImplementation#getClient(ServerName)} is called (Be sure to call
+   *         {@link Connection#close()} when done with this mocked Connection.
    */
-  public static ClusterConnection getMockedConnectionAndDecorate(final Configuration conf,
+  public static ConnectionImplementation getMockedConnectionAndDecorate(final Configuration conf,
       final AdminProtos.AdminService.BlockingInterface admin,
-      final ClientProtos.ClientService.BlockingInterface client,
-      final ServerName sn, final RegionInfo hri)
-  throws IOException {
+      final ClientProtos.ClientService.BlockingInterface client, final ServerName sn,
+      final RegionInfo hri) throws IOException {
     ConnectionImplementation c = Mockito.mock(ConnectionImplementation.class);
     Mockito.when(c.getConfiguration()).thenReturn(conf);
     Mockito.doNothing().when(c).close();
@@ -141,18 +133,17 @@ public class HConnectionTestingUtility {
   }
 
   /**
-   * Get a Mockito spied-upon {@link ClusterConnection} that goes with the passed
-   * <code>conf</code> configuration instance.
-   * Be sure to shutdown the connection when done by calling
-   * {@link Connection#close()} else it will stick around; this is probably not what you want.
+   * Get a Mockito spied-upon {@link ConnectionImplementation} that goes with the passed
+   * <code>conf</code> configuration instance. Be sure to shutdown the connection when done by
+   * calling {@link Connection#close()} else it will stick around; this is probably not what you
+   * want.
    * @param conf configuration
-   * @return ClusterConnection object for <code>conf</code>
-   * @throws ZooKeeperConnectionException
-   * [Dead link]: See also
-   * {http://mockito.googlecode.com/svn/branches/1.6/javadoc/org/mockito/Mockito.html#spy(T)}
+   * @return ConnectionImplementation object for <code>conf</code>
+   * @throws ZooKeeperConnectionException [Dead link]: See also
+   *           {http://mockito.googlecode.com/svn/branches/1.6/javadoc/org/mockito/Mockito.html#spy(T)}
    */
-  public static ClusterConnection getSpiedConnection(final Configuration conf)
-  throws IOException {
+  public static ConnectionImplementation getSpiedConnection(final Configuration conf)
+      throws IOException {
     ConnectionImplementation connection =
       Mockito.spy(new ConnectionImplementation(conf, null, null));
     return connection;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java
index 4e5665d..cd27a30 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java
@@ -111,8 +111,8 @@ public class TestCISleep extends AbstractTestCITimeout {
     }
 
     RegionAdminServiceCallable<Object> regionAdminServiceCallable =
-      new RegionAdminServiceCallable<Object>((ClusterConnection) TEST_UTIL.getConnection(),
-          new RpcControllerFactory(TEST_UTIL.getConfiguration()), tableName, FAM_NAM) {
+      new RegionAdminServiceCallable<Object>((ConnectionImplementation) TEST_UTIL.getConnection(),
+        new RpcControllerFactory(TEST_UTIL.getConfiguration()), tableName, FAM_NAM) {
         @Override
         public Object call(HBaseRpcController controller) throws Exception {
           return null;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
index a1026a9..ad73592 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
@@ -95,8 +95,9 @@ public class TestHBaseAdminNoCluster {
     configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, count);
     // Get mocked connection.   Getting the connection will register it so when HBaseAdmin is
     // constructed with same configuration, it will find this mocked connection.
-    ClusterConnection connection = HConnectionTestingUtility.getMockedConnection(configuration);
-    // Mock so we get back the master interface.  Make it so when createTable is called, we throw
+    ConnectionImplementation connection =
+      HConnectionTestingUtility.getMockedConnection(configuration);
+    // Mock so we get back the master interface. Make it so when createTable is called, we throw
     // the PleaseHoldException.
     MasterKeepAliveConnection masterAdmin = Mockito.mock(MasterKeepAliveConnection.class);
     Mockito.when(masterAdmin.createTable((RpcController)Mockito.any(),
@@ -292,7 +293,7 @@ public class TestHBaseAdminNoCluster {
     final int count = 10;
     configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, count);
 
-    ClusterConnection connection = mock(ClusterConnection.class);
+    ConnectionImplementation connection = mock(ConnectionImplementation.class);
     when(connection.getConfiguration()).thenReturn(configuration);
     MasterKeepAliveConnection masterAdmin =
         Mockito.mock(MasterKeepAliveConnection.class, new Answer() {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java
similarity index 86%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java
index 5d36ea9..53f5064 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase;
+package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -26,11 +26,19 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.NavigableMap;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.CellUtil;
+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.KeyValue;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -112,8 +120,8 @@ public class TestMetaTableAccessorNoCluster {
     assertTrue(hri == null);
     // OK, give it what it expects
     kvs.clear();
-    kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
-      HConstants.REGIONINFO_QUALIFIER, RegionInfo.toByteArray(RegionInfoBuilder.FIRST_META_REGIONINFO)));
+    kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f, HConstants.REGIONINFO_QUALIFIER,
+      RegionInfo.toByteArray(RegionInfoBuilder.FIRST_META_REGIONINFO)));
     hri = MetaTableAccessor.getRegionInfo(Result.create(kvs));
     assertNotNull(hri);
     assertTrue(RegionInfo.COMPARATOR.compare(hri, RegionInfoBuilder.FIRST_META_REGIONINFO) == 0);
@@ -123,8 +131,6 @@ public class TestMetaTableAccessorNoCluster {
    * Test that MetaTableAccessor will ride over server throwing
    * "Server not running" IOEs.
    * @see <a href="https://issues.apache.org/jira/browse/HBASE-3446">HBASE-3446</a>
-   * @throws IOException
-   * @throws InterruptedException
    */
   @Test
   public void testRideOverServerNotRunning()
@@ -135,7 +141,7 @@ public class TestMetaTableAccessorNoCluster {
     // This is a servername we use in a few places below.
     ServerName sn = ServerName.valueOf("example.com", 1234, System.currentTimeMillis());
 
-    ClusterConnection connection = null;
+    ConnectionImplementation connection = null;
     try {
       // Mock an ClientProtocol. Our mock implementation will fail a few
       // times when we go to open a scanner.
@@ -190,26 +196,27 @@ public class TestMetaTableAccessorNoCluster {
       // Return the RegionLocations object when locateRegion
       // The ugly format below comes of 'Important gotcha on spying real objects!' from
       // http://mockito.googlecode.com/svn/branches/1.6/javadoc/org/mockito/Mockito.html
-      Mockito.doReturn(rl).when
-      (connection).locateRegion((TableName)Mockito.any(), (byte[])Mockito.any(),
-              Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyInt());
+      Mockito.doReturn(rl).when(connection).locateRegion((TableName) Mockito.any(),
+        (byte[]) Mockito.any(), Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyInt());
 
       // Now shove our HRI implementation into the spied-upon connection.
-      Mockito.doReturn(implementation).
-        when(connection).getClient(Mockito.any());
+      Mockito.doReturn(implementation).when(connection).getClient(Mockito.any());
 
       // Scan meta for user tables and verify we got back expected answer.
       NavigableMap<RegionInfo, Result> hris =
         MetaTableAccessor.getServerUserRegions(connection, sn);
       assertEquals(1, hris.size());
-      assertTrue(RegionInfo.COMPARATOR.compare(hris.firstEntry().getKey(), RegionInfoBuilder.FIRST_META_REGIONINFO) == 0);
+      assertTrue(RegionInfo.COMPARATOR.compare(hris.firstEntry().getKey(),
+        RegionInfoBuilder.FIRST_META_REGIONINFO) == 0);
       assertTrue(Bytes.equals(rowToVerify, hris.firstEntry().getValue().getRow()));
       // Finally verify that scan was called four times -- three times
       // with exception and then on 4th attempt we succeed
-      Mockito.verify(implementation, Mockito.times(4)).
-        scan((RpcController)Mockito.any(), (ScanRequest)Mockito.any());
+      Mockito.verify(implementation, Mockito.times(4)).scan((RpcController) Mockito.any(),
+        (ScanRequest) Mockito.any());
     } finally {
-      if (connection != null && !connection.isClosed()) connection.close();
+      if (connection != null && !connection.isClosed()) {
+        connection.close();
+      }
       zkw.close();
     }
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 6c633a2..e5280a6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -253,7 +253,7 @@ public class TestMetaWithReplicas {
           util.getHBaseClusterInterface().killRegionServer(primary);
           util.getHBaseClusterInterface().waitForRegionServerToStop(primary, 60000);
         }
-        ((ClusterConnection)c).clearRegionCache();
+        c.clearRegionLocationCache();
       }
       LOG.info("Running GETs");
       Get get = null;
@@ -276,7 +276,7 @@ public class TestMetaWithReplicas {
         util.getHBaseClusterInterface().startRegionServer(primary.getHostname(), 0);
         util.getHBaseClusterInterface().waitForActiveAndReadyMaster();
         LOG.info("Master active!");
-        ((ClusterConnection)c).clearRegionCache();
+        c.clearRegionLocationCache();
       }
       conf.setBoolean(HConstants.USE_META_REPLICAS, false);
       LOG.info("Running GETs no replicas");
@@ -352,19 +352,24 @@ public class TestMetaWithReplicas {
     };
   }
 
+  private List<HRegionLocation> getMetaRegionLocations() throws IOException {
+    try (RegionLocator locator =
+      TEST_UTIL.getConnection().getRegionLocator(TableName.META_TABLE_NAME)) {
+      return locator.getAllRegionLocations();
+    }
+  }
+
   @Nullable
   private String checkMetaLocationAndExplain(int originalReplicaCount)
       throws KeeperException, IOException {
     List<String> metaZnodes = TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes();
     if (metaZnodes.size() == originalReplicaCount) {
-      RegionLocations rl = ((ClusterConnection) TEST_UTIL.getConnection())
-          .locateRegion(TableName.META_TABLE_NAME,
-              HConstants.EMPTY_START_ROW, false, false);
-      for (HRegionLocation location : rl.getRegionLocations()) {
+      List<HRegionLocation> locs = getMetaRegionLocations();
+      for (HRegionLocation location : locs) {
         if (location == null) {
-          return "Null location found in " + rl.toString();
+          return "Null location found in " + locs;
         }
-        if (location.getRegionInfo() == null) {
+        if (location.getRegion() == null) {
           return "Null regionInfo for location " + location;
         }
         if (location.getHostname() == null) {
@@ -387,8 +392,7 @@ public class TestMetaWithReplicas {
   public void testHBaseFsckWithFewerMetaReplicas() throws Exception {
     ClusterConnection c = (ClusterConnection)ConnectionFactory.createConnection(
         TEST_UTIL.getConfiguration());
-    RegionLocations rl = c.locateRegion(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW,
-        false, false);
+    RegionLocations rl = new RegionLocations(getMetaRegionLocations());
     HBaseFsckRepair.closeRegionSilentlyAndWait(c,
         rl.getRegionLocation(1).getServerName(), rl.getRegionLocation(1).getRegionInfo());
     // check that problem exists
@@ -405,8 +409,7 @@ public class TestMetaWithReplicas {
   public void testHBaseFsckWithFewerMetaReplicaZnodes() throws Exception {
     ClusterConnection c = (ClusterConnection)ConnectionFactory.createConnection(
         TEST_UTIL.getConfiguration());
-    RegionLocations rl = c.locateRegion(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW,
-        false, false);
+    RegionLocations rl = new RegionLocations(getMetaRegionLocations());
     HBaseFsckRepair.closeRegionSilentlyAndWait(c,
         rl.getRegionLocation(2).getServerName(), rl.getRegionLocation(2).getRegionInfo());
     ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
@@ -484,17 +487,14 @@ public class TestMetaWithReplicas {
     // can be recovered
     try (ClusterConnection conn = (ClusterConnection)
         ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
-      RegionLocations rl = conn.
-          locateRegion(TableName.META_TABLE_NAME, Bytes.toBytes(""), false, true);
-      HRegionLocation hrl = rl.getRegionLocation(1);
+      HRegionLocation hrl = getMetaRegionLocations().get(1);
       ServerName oldServer = hrl.getServerName();
       TEST_UTIL.getHBaseClusterInterface().killRegionServer(oldServer);
       int i = 0;
       do {
         LOG.debug("Waiting for the replica " + hrl.getRegionInfo() + " to come up");
         Thread.sleep(10000); //wait for the detection/recovery
-        rl = conn.locateRegion(TableName.META_TABLE_NAME, Bytes.toBytes(""), false, true);
-        hrl = rl.getRegionLocation(1);
+        hrl = getMetaRegionLocations().get(1);
         i++;
       } while ((hrl == null || hrl.getServerName().equals(oldServer)) && i < 3);
       assertTrue(i != 3);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index a42b26d..987ac7e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -670,9 +670,9 @@ public class TestReplicaWithCluster {
   // within configured hbase.client.metaReplicaCallTimeout.scan from primary meta region.
   @Test
   public void testGetRegionLocationFromPrimaryMetaRegion() throws IOException, InterruptedException {
-    HTU.getAdmin().setBalancerRunning(false, true);
+    HTU.getAdmin().balancerSwitch(false, true);
 
-    ((ConnectionImplementation) HTU.getAdmin().getConnection()).setUseMetaReplicas(true);
+    ((ConnectionImplementation) HTU.getConnection()).setUseMetaReplicas(true);
 
     // Create table then get the single region for our new table.
     HTableDescriptor hdt = HTU.createTableDescriptor("testGetRegionLocationFromPrimaryMetaRegion");
@@ -684,13 +684,13 @@ public class TestReplicaWithCluster {
       RegionServerHostingPrimayMetaRegionSlowOrStopCopro.slowDownPrimaryMetaScan = true;
 
       // Get user table location, always get it from the primary meta replica
-      RegionLocations url = ((ClusterConnection) HTU.getConnection())
-          .locateRegion(hdt.getTableName(), row, false, false);
-
+      try (RegionLocator locator = HTU.getConnection().getRegionLocator(hdt.getTableName())) {
+        locator.getRegionLocations(row, true);
+      }
     } finally {
       RegionServerHostingPrimayMetaRegionSlowOrStopCopro.slowDownPrimaryMetaScan = false;
-      ((ConnectionImplementation) HTU.getAdmin().getConnection()).setUseMetaReplicas(false);
-      HTU.getAdmin().setBalancerRunning(true, true);
+      ((ConnectionImplementation) HTU.getConnection()).setUseMetaReplicas(false);
+      HTU.getAdmin().balancerSwitch(true, true);
       HTU.getAdmin().disableTable(hdt.getTableName());
       HTU.deleteTable(hdt.getTableName());
     }
@@ -703,25 +703,27 @@ public class TestReplicaWithCluster {
   // with the primary meta region.
   @Test
   public void testReplicaGetWithPrimaryAndMetaDown() throws IOException, InterruptedException {
-    HTU.getAdmin().setBalancerRunning(false, true);
+    HTU.getAdmin().balancerSwitch(false, true);
 
-    ((ConnectionImplementation)HTU.getAdmin().getConnection()).setUseMetaReplicas(true);
+    ((ConnectionImplementation)HTU.getConnection()).setUseMetaReplicas(true);
 
     // Create table then get the single region for our new table.
     HTableDescriptor hdt = HTU.createTableDescriptor("testReplicaGetWithPrimaryAndMetaDown");
     hdt.setRegionReplication(2);
     try {
-
       Table table = HTU.createTable(hdt, new byte[][] { f }, null);
-
       // Get Meta location
-      RegionLocations mrl = ((ClusterConnection) HTU.getConnection())
-          .locateRegion(TableName.META_TABLE_NAME,
-              HConstants.EMPTY_START_ROW, false, false);
+      RegionLocations mrl;
+      try (
+          RegionLocator locator = HTU.getConnection().getRegionLocator(TableName.META_TABLE_NAME)) {
+        mrl = new RegionLocations(locator.getRegionLocations(HConstants.EMPTY_START_ROW, true));
+      }
 
       // Get user table location
-      RegionLocations url = ((ClusterConnection) HTU.getConnection())
-          .locateRegion(hdt.getTableName(), row, false, false);
+      RegionLocations url;
+      try (RegionLocator locator = HTU.getConnection().getRegionLocator(hdt.getTableName())) {
+        url = new RegionLocations(locator.getRegionLocations(row, true));
+      }
 
       // Make sure that user primary region is co-hosted with the meta region
       if (!url.getDefaultRegionLocation().getServerName().equals(
@@ -740,12 +742,15 @@ public class TestReplicaWithCluster {
 
       // Wait until the meta table is updated with new location info
       while (true) {
-        mrl = ((ClusterConnection) HTU.getConnection())
-            .locateRegion(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW, false, false);
+        try (RegionLocator locator =
+          HTU.getConnection().getRegionLocator(TableName.META_TABLE_NAME)) {
+          mrl = new RegionLocations(locator.getRegionLocations(HConstants.EMPTY_START_ROW, true));
+        }
 
         // Get user table location
-        url = ((ClusterConnection) HTU.getConnection())
-            .locateRegion(hdt.getTableName(), row, false, true);
+        try (RegionLocator locator = HTU.getConnection().getRegionLocator(hdt.getTableName())) {
+          url = new RegionLocations(locator.getRegionLocations(row, true));
+        }
 
         LOG.info("meta locations " + mrl);
         LOG.info("table locations " + url);
@@ -787,9 +792,9 @@ public class TestReplicaWithCluster {
       Assert.assertTrue(r.isStale());
 
     } finally {
-      ((ConnectionImplementation)HTU.getAdmin().getConnection()).setUseMetaReplicas(false);
+      ((ConnectionImplementation)HTU.getConnection()).setUseMetaReplicas(false);
       RegionServerHostingPrimayMetaRegionSlowOrStopCopro.throwException = false;
-      HTU.getAdmin().setBalancerRunning(true, true);
+      HTU.getAdmin().balancerSwitch(true, true);
       HTU.getAdmin().disableTable(hdt.getTableName());
       HTU.deleteTable(hdt.getTableName());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
index 4a72410..6616b3b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
@@ -229,7 +229,7 @@ public class TestReplicasClient {
 
   @Before
   public void before() throws IOException {
-    ((ClusterConnection) HTU.getAdmin().getConnection()).clearRegionCache();
+    HTU.getConnection().clearRegionLocationCache();
     try {
       openRegion(hriPrimary);
     } catch (Exception ignored) {
@@ -250,8 +250,7 @@ public class TestReplicasClient {
       closeRegion(hriPrimary);
     } catch (Exception ignored) {
     }
-
-    ((ClusterConnection) HTU.getAdmin().getConnection()).clearRegionCache();
+    HTU.getConnection().clearRegionLocationCache();
   }
 
   private HRegionServer getRS() {
@@ -329,17 +328,17 @@ public class TestReplicasClient {
   public void testLocations() throws Exception {
     byte[] b1 = Bytes.toBytes("testLocations");
     openRegion(hriSecondary);
-    ClusterConnection hc = (ClusterConnection) HTU.getAdmin().getConnection();
+    ConnectionImplementation hc = (ConnectionImplementation) HTU.getConnection();
 
     try {
-      hc.clearRegionCache();
+      hc.clearRegionLocationCache();
       RegionLocations rl = hc.locateRegion(table.getName(), b1, false, false);
       Assert.assertEquals(2, rl.size());
 
       rl = hc.locateRegion(table.getName(), b1, true, false);
       Assert.assertEquals(2, rl.size());
 
-      hc.clearRegionCache();
+      hc.clearRegionLocationCache();
       rl = hc.locateRegion(table.getName(), b1, true, false);
       Assert.assertEquals(2, rl.size());
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
index 026010d..93c6b4d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
@@ -204,7 +204,7 @@ public class TestSeparateClientZKCluster {
   public void testMetaMoveDuringClientZkClusterRestart() throws Exception {
     TableName tn = TableName.valueOf(name.getMethodName());
     // create table
-    ClusterConnection conn = (ClusterConnection) TEST_UTIL.getConnection();
+    Connection conn = TEST_UTIL.getConnection();
     Admin admin = conn.getAdmin();
     HTable table = (HTable) conn.getTable(tn);
     try {
@@ -218,7 +218,7 @@ public class TestSeparateClientZKCluster {
       put.addColumn(family, qualifier, value);
       table.put(put);
       // invalid connection cache
-      conn.clearRegionCache();
+      conn.clearRegionLocationCache();
       // stop client zk cluster
       clientZkCluster.shutdown();
       // stop current meta server and confirm the server shutdown process
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
index 0562c90..2f95a90 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
@@ -297,7 +297,7 @@ public class TestSnapshotCloneIndependence {
    */
   private void runTestRegionOperationsIndependent() throws Exception {
     // Verify that region information is the same pre-split
-    ((ClusterConnection) UTIL.getConnection()).clearRegionCache();
+    UTIL.getConnection().clearRegionLocationCache();
     List<HRegionInfo> originalTableHRegions = admin.getTableRegions(originalTableName);
 
     final int originalRegionCount = originalTableHRegions.size();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java
index 446c3f9..0c2532e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java
@@ -25,8 +25,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -58,32 +59,33 @@ public class TestMetaAssignmentWithStopMaster {
 
   @Test
   public void testStopActiveMaster() throws Exception {
-    ClusterConnection conn =
-        (ClusterConnection) ConnectionFactory.createConnection(UTIL.getConfiguration());
-    ServerName oldMetaServer = conn.locateRegions(TableName.META_TABLE_NAME).get(0).getServerName();
-    ServerName oldMaster = UTIL.getMiniHBaseCluster().getMaster().getServerName();
+    try (Connection conn = ConnectionFactory.createConnection(UTIL.getConfiguration());
+        RegionLocator locator = conn.getRegionLocator(TableName.META_TABLE_NAME)) {
+      ServerName oldMetaServer = locator.getAllRegionLocations().get(0).getServerName();
+      ServerName oldMaster = UTIL.getMiniHBaseCluster().getMaster().getServerName();
 
-    UTIL.getMiniHBaseCluster().getMaster().stop("Stop master for test");
-    long startTime = System.currentTimeMillis();
-    while (UTIL.getMiniHBaseCluster().getMaster() == null || UTIL.getMiniHBaseCluster().getMaster()
-        .getServerName().equals(oldMaster)) {
-      LOG.info("Wait the standby master become active");
-      Thread.sleep(3000);
-      if (System.currentTimeMillis() - startTime > WAIT_TIMEOUT) {
-        fail("Wait too long for standby master become active");
+      UTIL.getMiniHBaseCluster().getMaster().stop("Stop master for test");
+      long startTime = System.currentTimeMillis();
+      while (UTIL.getMiniHBaseCluster().getMaster() == null ||
+        UTIL.getMiniHBaseCluster().getMaster().getServerName().equals(oldMaster)) {
+        LOG.info("Wait the standby master become active");
+        Thread.sleep(3000);
+        if (System.currentTimeMillis() - startTime > WAIT_TIMEOUT) {
+          fail("Wait too long for standby master become active");
+        }
       }
-    }
-    startTime = System.currentTimeMillis();
-    while (!UTIL.getMiniHBaseCluster().getMaster().isInitialized()) {
-      LOG.info("Wait the new active master to be initialized");
-      Thread.sleep(3000);
-      if (System.currentTimeMillis() - startTime > WAIT_TIMEOUT) {
-        fail("Wait too long for the new active master to be initialized");
+      startTime = System.currentTimeMillis();
+      while (!UTIL.getMiniHBaseCluster().getMaster().isInitialized()) {
+        LOG.info("Wait the new active master to be initialized");
+        Thread.sleep(3000);
+        if (System.currentTimeMillis() - startTime > WAIT_TIMEOUT) {
+          fail("Wait too long for the new active master to be initialized");
+        }
       }
-    }
 
-    ServerName newMetaServer = conn.locateRegions(TableName.META_TABLE_NAME).get(0).getServerName();
-    assertTrue("The new meta server " + newMetaServer + " should be same with" +
+      ServerName newMetaServer = locator.getAllRegionLocations().get(0).getServerName();
+      assertTrue("The new meta server " + newMetaServer + " should be same with" +
         " the old meta server " + oldMetaServer, newMetaServer.equals(oldMetaServer));
+    }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java
index a4b99a1..fcc1bb8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.fs.Path;
 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.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
@@ -50,6 +49,7 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 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.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionLocator;
@@ -366,24 +366,15 @@ public class TestLoadIncrementalHFilesSplitRecovery {
 
   private ClusterConnection getMockedConnection(final Configuration conf)
       throws IOException, org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
-    ClusterConnection c = Mockito.mock(ClusterConnection.class);
-    Mockito.when(c.getConfiguration()).thenReturn(conf);
-    Mockito.doNothing().when(c).close();
-    // Make it so we return a particular location when asked.
-    final HRegionLocation loc = new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO,
-        ServerName.valueOf("example.org", 1234, 0));
-    Mockito.when(
-      c.getRegionLocation((TableName) Mockito.any(), (byte[]) Mockito.any(), Mockito.anyBoolean()))
-        .thenReturn(loc);
-    Mockito.when(c.locateRegion((TableName) Mockito.any(), (byte[]) Mockito.any())).thenReturn(loc);
-    ClientProtos.ClientService.BlockingInterface hri =
-        Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
+    ServerName sn = ServerName.valueOf("example.org", 1234, 0);
+    RegionInfo hri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+    ClientProtos.ClientService.BlockingInterface client =
+      Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
     Mockito
-        .when(
-          hri.bulkLoadHFile((RpcController) Mockito.any(), (BulkLoadHFileRequest) Mockito.any()))
-        .thenThrow(new ServiceException(new IOException("injecting bulk load error")));
-    Mockito.when(c.getClient(Mockito.any())).thenReturn(hri);
-    return c;
+      .when(
+        client.bulkLoadHFile((RpcController) Mockito.any(), (BulkLoadHFileRequest) Mockito.any()))
+      .thenThrow(new ServiceException(new IOException("injecting bulk load error")));
+    return HConnectionTestingUtility.getMockedConnectionAndDecorate(conf, null, client, sn, hri);
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index fc23d51..d25ccef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
@@ -307,8 +307,7 @@ public class BaseTestHBaseFsck {
       tbl.close();
       tbl = null;
     }
-
-    ((ClusterConnection) connection).clearRegionCache();
+    connection.clearRegionLocationCache();
     deleteTable(TEST_UTIL, tablename);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
index 410dd0c..0a66ec0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
@@ -19,33 +19,34 @@ package org.apache.hadoop.hbase.util;
 import static org.apache.hadoop.hbase.util.test.LoadTestDataGenerator.INCREMENT;
 import static org.apache.hadoop.hbase.util.test.LoadTestDataGenerator.MUTATE_INFO;
 
-import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
-
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
 import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
+
 /**
  * Common base class for reader and writer parts of multi-thread HBase load
  * test (See LoadTestTool).
@@ -491,7 +492,6 @@ public abstract class MultiThreadedAction {
   }
 
   private void printLocations(Result r) {
-    RegionLocations rl = null;
     if (r == null) {
       LOG.info("FAILED FOR null Result");
       return;
@@ -500,15 +500,14 @@ public abstract class MultiThreadedAction {
     if (r.getRow() == null) {
       return;
     }
-    try {
-      rl = ((ClusterConnection)connection).locateRegion(tableName, r.getRow(), true, true);
+    try (RegionLocator locator = connection.getRegionLocator(tableName)) {
+      List<HRegionLocation> locs = locator.getRegionLocations(r.getRow());
+      for (HRegionLocation h : locs) {
+        LOG.info("LOCATION " + h);
+      }
     } catch (IOException e) {
       LOG.warn("Couldn't get locations for row " + Bytes.toString(r.getRow()));
     }
-    HRegionLocation locations[] = rl.getRegionLocations();
-    for (HRegionLocation h : locations) {
-      LOG.info("LOCATION " + h);
-    }
   }
 
   private String resultToString(Result result) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java
index 6864366..27f5fb9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java
@@ -21,14 +21,13 @@ import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Get;
-
 import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
@@ -377,8 +376,10 @@ public class MultiThreadedReader extends MultiThreadedAction
           numKeysVerified.incrementAndGet();
         }
       } else {
-        HRegionLocation hloc = connection.getRegionLocation(tableName,
-          get.getRow(), false);
+        HRegionLocation hloc;
+        try (RegionLocator locator = connection.getRegionLocator(tableName)) {
+          hloc = locator.getRegionLocation(get.getRow());
+        }
         String rowKey = Bytes.toString(get.getRow());
         LOG.info("Key = " + rowKey + ", Region location: " + hloc);
         if(isNullExpected) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterBase.java
index 54be0d3..1ebc9b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterBase.java
@@ -27,10 +27,10 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
 import org.slf4j.Logger;
@@ -97,9 +97,9 @@ public abstract class MultiThreadedWriterBase extends MultiThreadedAction {
 
   protected String getRegionDebugInfoSafe(Table table, byte[] rowKey) {
     HRegionLocation cached = null, real = null;
-    try {
-      cached = connection.getRegionLocation(tableName, rowKey, false);
-      real = connection.getRegionLocation(tableName, rowKey, true);
+    try (RegionLocator locator = connection.getRegionLocator(tableName)) {
+      cached = locator.getRegionLocation(rowKey, false);
+      real = locator.getRegionLocation(rowKey, true);
     } catch (Throwable t) {
       // Cannot obtain region information for another catch block - too bad!
     }


[hbase] 04/11: HBASE-21579 Use AsyncClusterConnection for HBaseInterClusterReplicationEndpoint

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

zhangduo pushed a commit to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 247a3b89df1fbdaa94476432478c049864ff70dc
Author: zhangduo <zh...@apache.org>
AuthorDate: Tue Jan 1 21:27:14 2019 +0800

    HBASE-21579 Use AsyncClusterConnection for HBaseInterClusterReplicationEndpoint
---
 .../hbase/client/AsyncRegionServerAdmin.java       | 14 +++++---
 .../hbase/protobuf/ReplicationProtbufUtil.java     | 35 ++++++++++---------
 .../HBaseInterClusterReplicationEndpoint.java      | 31 +++++++++--------
 .../regionserver/ReplicationSinkManager.java       | 40 ++++++++--------------
 .../hbase/replication/SyncReplicationTestBase.java | 12 +++----
 .../regionserver/TestReplicationSinkManager.java   | 21 +++++-------
 6 files changed, 74 insertions(+), 79 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
index 9accd89..b9141a9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -94,9 +95,9 @@ public class AsyncRegionServerAdmin {
     void call(AdminService.Interface stub, HBaseRpcController controller, RpcCallback<RESP> done);
   }
 
-  private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall) {
+  private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall, CellScanner cellScanner) {
     CompletableFuture<RESP> future = new CompletableFuture<>();
-    HBaseRpcController controller = conn.rpcControllerFactory.newController();
+    HBaseRpcController controller = conn.rpcControllerFactory.newController(cellScanner);
     try {
       rpcCall.call(conn.getAdminStub(server), controller, new RpcCallback<RESP>() {
 
@@ -115,6 +116,10 @@ public class AsyncRegionServerAdmin {
     return future;
   }
 
+  private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall) {
+    return call(rpcCall, null);
+  }
+
   public CompletableFuture<GetRegionInfoResponse> getRegionInfo(GetRegionInfoRequest request) {
     return call((stub, controller, done) -> stub.getRegionInfo(controller, request, done));
   }
@@ -154,8 +159,9 @@ public class AsyncRegionServerAdmin {
   }
 
   public CompletableFuture<ReplicateWALEntryResponse> replicateWALEntry(
-      ReplicateWALEntryRequest request) {
-    return call((stub, controller, done) -> stub.replicateWALEntry(controller, request, done));
+      ReplicateWALEntryRequest request, CellScanner cellScanner) {
+    return call((stub, controller, done) -> stub.replicateWALEntry(controller, request, done),
+      cellScanner);
   }
 
   public CompletableFuture<ReplicateWALEntryResponse> replay(ReplicateWALEntryRequest request) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
index c1b3911..74fad26 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
@@ -20,51 +20,54 @@ package org.apache.hadoop.hbase.protobuf;
 
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.io.SizedCellScanner;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 
 @InterfaceAudience.Private
 public class ReplicationProtbufUtil {
+
   /**
-   * A helper to replicate a list of WAL entries using admin protocol.
-   * @param admin Admin service
+   * A helper to replicate a list of WAL entries using region server admin
+   * @param admin the region server admin
    * @param entries Array of WAL entries to be replicated
    * @param replicationClusterId Id which will uniquely identify source cluster FS client
    *          configurations in the replication configuration directory
    * @param sourceBaseNamespaceDir Path to source cluster base namespace directory
    * @param sourceHFileArchiveDir Path to the source cluster hfile archive directory
-   * @throws java.io.IOException
    */
-  public static void replicateWALEntry(final AdminService.BlockingInterface admin,
-      final Entry[] entries, String replicationClusterId, Path sourceBaseNamespaceDir,
-      Path sourceHFileArchiveDir) throws IOException {
-    Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
-        buildReplicateWALEntryRequest(entries, null, replicationClusterId, sourceBaseNamespaceDir,
-          sourceHFileArchiveDir);
-    HBaseRpcController controller = new HBaseRpcControllerImpl(p.getSecond());
+  public static void replicateWALEntry(AsyncRegionServerAdmin admin, Entry[] entries,
+      String replicationClusterId, Path sourceBaseNamespaceDir, Path sourceHFileArchiveDir)
+      throws IOException {
+    Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p = buildReplicateWALEntryRequest(
+      entries, null, replicationClusterId, sourceBaseNamespaceDir, sourceHFileArchiveDir);
     try {
-      admin.replicateWALEntry(controller, p.getFirst());
-    } catch (org.apache.hbase.thirdparty.com.google.protobuf.ServiceException e) {
-      throw ProtobufUtil.getServiceException(e);
+      admin.replicateWALEntry(p.getFirst(), p.getSecond()).get();
+    } catch (InterruptedException e) {
+      throw (IOException) new InterruptedIOException().initCause(e);
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      Throwables.propagateIfPossible(cause, IOException.class);
+      throw new IOException(e);
     }
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index 57301fc..81ed4b8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -39,7 +39,6 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -48,13 +47,16 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
+import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
@@ -65,8 +67,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
-
 /**
  * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint}
  * implementation for replicating to another HBase cluster.
@@ -85,8 +85,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
 
   private static final long DEFAULT_MAX_TERMINATION_WAIT_MULTIPLIER = 2;
 
-  private ClusterConnection conn;
-  private Configuration localConf;
+  private AsyncClusterConnection conn;
   private Configuration conf;
   // How long should we sleep for each retry
   private long sleepForRetries;
@@ -117,7 +116,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
   public void init(Context context) throws IOException {
     super.init(context);
     this.conf = HBaseConfiguration.create(ctx.getConfiguration());
-    this.localConf = HBaseConfiguration.create(ctx.getLocalConfiguration());
     decorateConf();
     this.maxRetriesMultiplier = this.conf.getInt("replication.source.maxretriesmultiplier", 300);
     this.socketTimeoutMultiplier = this.conf.getInt("replication.source.socketTimeoutMultiplier",
@@ -132,12 +130,13 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
     // TODO: This connection is replication specific or we should make it particular to
     // replication and make replication specific settings such as compression or codec to use
     // passing Cells.
-    this.conn = (ClusterConnection) ConnectionFactory.createConnection(this.conf);
+    this.conn =
+      ClusterConnectionFactory.createAsyncClusterConnection(conf, null, User.getCurrent());
     this.sleepForRetries =
         this.conf.getLong("replication.source.sleepforretries", 1000);
     this.metrics = context.getMetrics();
     // ReplicationQueueInfo parses the peerId out of the znode for us
-    this.replicationSinkMgr = new ReplicationSinkManager(conn, ctx.getPeerId(), this, this.conf);
+    this.replicationSinkMgr = new ReplicationSinkManager(conn, this, this.conf);
     // per sink thread pool
     this.maxThreads = this.conf.getInt(HConstants.REPLICATION_SOURCE_MAXTHREADS_KEY,
       HConstants.REPLICATION_SOURCE_MAXTHREADS_DEFAULT);
@@ -284,9 +283,10 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
   }
 
   private void reconnectToPeerCluster() {
-    ClusterConnection connection = null;
+    AsyncClusterConnection connection = null;
     try {
-      connection = (ClusterConnection) ConnectionFactory.createConnection(this.conf);
+      connection =
+        ClusterConnectionFactory.createAsyncClusterConnection(conf, null, User.getCurrent());
     } catch (IOException ioe) {
       LOG.warn("Failed to create connection for peer cluster", ioe);
     }
@@ -366,7 +366,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
         }
         continue;
       }
-      if (this.conn == null || this.conn.isClosed()) {
+      if (this.conn == null) {
         reconnectToPeerCluster();
       }
       try {
@@ -470,10 +470,11 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
           entriesHashCode, entries.size(), size, replicationClusterId);
       }
       sinkPeer = replicationSinkMgr.getReplicationSink();
-      BlockingInterface rrs = sinkPeer.getRegionServer();
+      AsyncRegionServerAdmin rsAdmin = sinkPeer.getRegionServer();
       try {
-        ReplicationProtbufUtil.replicateWALEntry(rrs, entries.toArray(new Entry[entries.size()]),
-          replicationClusterId, baseNamespaceDir, hfileArchiveDir);
+        ReplicationProtbufUtil.replicateWALEntry(rsAdmin,
+          entries.toArray(new Entry[entries.size()]), replicationClusterId, baseNamespaceDir,
+          hfileArchiveDir);
         LOG.trace("Completed replicating batch {}", entriesHashCode);
       } catch (IOException e) {
         LOG.trace("Failed replicating batch {}", entriesHashCode, e);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
index 3cd7884..21b07ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
@@ -21,11 +21,11 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -35,8 +35,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-
 /**
  * Maintains a collection of peers to replicate to, and randomly selects a
  * single peer to replicate to per set of data to replicate. Also handles
@@ -61,9 +59,7 @@ public class ReplicationSinkManager {
   static final float DEFAULT_REPLICATION_SOURCE_RATIO = 0.5f;
 
 
-  private final Connection conn;
-
-  private final String peerClusterId;
+  private final AsyncClusterConnection conn;
 
   private final HBaseReplicationEndpoint endpoint;
 
@@ -77,8 +73,6 @@ public class ReplicationSinkManager {
   // replication sinks is refreshed
   private final int badSinkThreshold;
 
-  private final Random random;
-
   // A timestamp of the last time the list of replication peers changed
   private long lastUpdateToPeers;
 
@@ -88,26 +82,22 @@ public class ReplicationSinkManager {
   /**
    * Instantiate for a single replication peer cluster.
    * @param conn connection to the peer cluster
-   * @param peerClusterId identifier of the peer cluster
    * @param endpoint replication endpoint for inter cluster replication
    * @param conf HBase configuration, used for determining replication source ratio and bad peer
    *          threshold
    */
-  public ReplicationSinkManager(ClusterConnection conn, String peerClusterId,
-      HBaseReplicationEndpoint endpoint, Configuration conf) {
+  public ReplicationSinkManager(AsyncClusterConnection conn, HBaseReplicationEndpoint endpoint,
+      Configuration conf) {
     this.conn = conn;
-    this.peerClusterId = peerClusterId;
     this.endpoint = endpoint;
     this.badReportCounts = Maps.newHashMap();
     this.ratio = conf.getFloat("replication.source.ratio", DEFAULT_REPLICATION_SOURCE_RATIO);
-    this.badSinkThreshold = conf.getInt("replication.bad.sink.threshold",
-                                        DEFAULT_BAD_SINK_THRESHOLD);
-    this.random = new Random();
+    this.badSinkThreshold =
+      conf.getInt("replication.bad.sink.threshold", DEFAULT_BAD_SINK_THRESHOLD);
   }
 
   /**
    * Get a randomly-chosen replication sink to replicate to.
-   *
    * @return a replication sink to replicate to
    */
   public synchronized SinkPeer getReplicationSink() throws IOException {
@@ -119,8 +109,8 @@ public class ReplicationSinkManager {
     if (sinks.isEmpty()) {
       throw new IOException("No replication sinks are available");
     }
-    ServerName serverName = sinks.get(random.nextInt(sinks.size()));
-    return new SinkPeer(serverName, ((ClusterConnection) conn).getAdmin(serverName));
+    ServerName serverName = sinks.get(ThreadLocalRandom.current().nextInt(sinks.size()));
+    return new SinkPeer(serverName, conn.getRegionServerAdmin(serverName));
   }
 
   /**
@@ -160,7 +150,7 @@ public class ReplicationSinkManager {
    */
   public synchronized void chooseSinks() {
     List<ServerName> slaveAddresses = endpoint.getRegionServers();
-    Collections.shuffle(slaveAddresses, random);
+    Collections.shuffle(slaveAddresses, ThreadLocalRandom.current());
     int numSinks = (int) Math.ceil(slaveAddresses.size() * ratio);
     sinks = slaveAddresses.subList(0, numSinks);
     lastUpdateToPeers = System.currentTimeMillis();
@@ -182,9 +172,9 @@ public class ReplicationSinkManager {
    */
   public static class SinkPeer {
     private ServerName serverName;
-    private AdminService.BlockingInterface regionServer;
+    private AsyncRegionServerAdmin regionServer;
 
-    public SinkPeer(ServerName serverName, AdminService.BlockingInterface regionServer) {
+    public SinkPeer(ServerName serverName, AsyncRegionServerAdmin regionServer) {
       this.serverName = serverName;
       this.regionServer = regionServer;
     }
@@ -193,10 +183,8 @@ public class ReplicationSinkManager {
       return serverName;
     }
 
-    public AdminService.BlockingInterface getRegionServer() {
+    public AsyncRegionServerAdmin getRegionServer() {
       return regionServer;
     }
-
   }
-
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
index f373590..e0d112d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -250,19 +250,19 @@ public class SyncReplicationTestBase {
   protected final void verifyReplicationRequestRejection(HBaseTestingUtility utility,
       boolean expectedRejection) throws Exception {
     HRegionServer regionServer = utility.getRSForFirstRegionInTable(TABLE_NAME);
-    ClusterConnection connection = regionServer.getClusterConnection();
+    AsyncClusterConnection connection = regionServer.getAsyncClusterConnection();
     Entry[] entries = new Entry[10];
     for (int i = 0; i < entries.length; i++) {
       entries[i] =
         new Entry(new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, TABLE_NAME, 0), new WALEdit());
     }
     if (!expectedRejection) {
-      ReplicationProtbufUtil.replicateWALEntry(connection.getAdmin(regionServer.getServerName()),
-        entries, null, null, null);
+      ReplicationProtbufUtil.replicateWALEntry(
+        connection.getRegionServerAdmin(regionServer.getServerName()), entries, null, null, null);
     } else {
       try {
-        ReplicationProtbufUtil.replicateWALEntry(connection.getAdmin(regionServer.getServerName()),
-          entries, null, null, null);
+        ReplicationProtbufUtil.replicateWALEntry(
+          connection.getRegionServerAdmin(regionServer.getServerName()), entries, null, null, null);
         fail("Should throw IOException when sync-replication state is in A or DA");
       } catch (DoNotRetryIOException e) {
         assertTrue(e.getMessage().contains("Reject to apply to sink cluster"));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
index 39dabb4..60afd40 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
@@ -25,7 +25,8 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -37,8 +38,6 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-
 @Category({ReplicationTests.class, SmallTests.class})
 public class TestReplicationSinkManager {
 
@@ -46,16 +45,14 @@ public class TestReplicationSinkManager {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestReplicationSinkManager.class);
 
-  private static final String PEER_CLUSTER_ID = "PEER_CLUSTER_ID";
-
   private HBaseReplicationEndpoint replicationEndpoint;
   private ReplicationSinkManager sinkManager;
 
   @Before
   public void setUp() {
     replicationEndpoint = mock(HBaseReplicationEndpoint.class);
-    sinkManager = new ReplicationSinkManager(mock(ClusterConnection.class),
-                      PEER_CLUSTER_ID, replicationEndpoint, new Configuration());
+    sinkManager = new ReplicationSinkManager(mock(AsyncClusterConnection.class),
+      replicationEndpoint, new Configuration());
   }
 
   @Test
@@ -100,7 +97,7 @@ public class TestReplicationSinkManager {
     // Sanity check
     assertEquals(1, sinkManager.getNumSinks());
 
-    SinkPeer sinkPeer = new SinkPeer(serverNameA, mock(AdminService.BlockingInterface.class));
+    SinkPeer sinkPeer = new SinkPeer(serverNameA, mock(AsyncRegionServerAdmin.class));
 
     sinkManager.reportBadSink(sinkPeer);
 
@@ -131,7 +128,7 @@ public class TestReplicationSinkManager {
 
     ServerName serverName = sinkManager.getSinksForTesting().get(0);
 
-    SinkPeer sinkPeer = new SinkPeer(serverName, mock(AdminService.BlockingInterface.class));
+    SinkPeer sinkPeer = new SinkPeer(serverName, mock(AsyncRegionServerAdmin.class));
 
     sinkManager.reportSinkSuccess(sinkPeer); // has no effect, counter does not go negative
     for (int i = 0; i <= ReplicationSinkManager.DEFAULT_BAD_SINK_THRESHOLD; i++) {
@@ -147,7 +144,7 @@ public class TestReplicationSinkManager {
     //
     serverName = sinkManager.getSinksForTesting().get(0);
 
-    sinkPeer = new SinkPeer(serverName, mock(AdminService.BlockingInterface.class));
+    sinkPeer = new SinkPeer(serverName, mock(AsyncRegionServerAdmin.class));
     for (int i = 0; i <= ReplicationSinkManager.DEFAULT_BAD_SINK_THRESHOLD-1; i++) {
       sinkManager.reportBadSink(sinkPeer);
     }
@@ -188,8 +185,8 @@ public class TestReplicationSinkManager {
     ServerName serverNameA = sinkList.get(0);
     ServerName serverNameB = sinkList.get(1);
 
-    SinkPeer sinkPeerA = new SinkPeer(serverNameA, mock(AdminService.BlockingInterface.class));
-    SinkPeer sinkPeerB = new SinkPeer(serverNameB, mock(AdminService.BlockingInterface.class));
+    SinkPeer sinkPeerA = new SinkPeer(serverNameA, mock(AsyncRegionServerAdmin.class));
+    SinkPeer sinkPeerB = new SinkPeer(serverNameB, mock(AsyncRegionServerAdmin.class));
 
     for (int i = 0; i <= ReplicationSinkManager.DEFAULT_BAD_SINK_THRESHOLD; i++) {
       sinkManager.reportBadSink(sinkPeerA);


[hbase] 02/11: HBASE-21516 Use AsyncConnection instead of Connection in SecureBulkLoadManager

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

zhangduo pushed a commit to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 5564ddd29fac0a9bef13cf3fcf068d4c6b3656d0
Author: zhangduo <zh...@apache.org>
AuthorDate: Sat Dec 1 21:15:48 2018 +0800

    HBASE-21516 Use AsyncConnection instead of Connection in SecureBulkLoadManager
---
 .../apache/hadoop/hbase/protobuf/ProtobufUtil.java |  5 +-
 .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java |  7 ++-
 .../hadoop/hbase/regionserver/HRegionServer.java   |  2 +-
 .../hbase/regionserver/SecureBulkLoadManager.java  | 24 ++++-----
 .../hadoop/hbase/security/token/TokenUtil.java     | 57 +++++++++++++++++-----
 .../hadoop/hbase/security/token/TestTokenUtil.java | 42 ++++++++++++----
 6 files changed, 96 insertions(+), 41 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index a3d49b5..d9e620b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -261,13 +261,12 @@ public final class ProtobufUtil {
    * just {@link ServiceException}. Prefer this method to
    * {@link #getRemoteException(ServiceException)} because trying to
    * contain direct protobuf references.
-   * @param e
    */
-  public static IOException handleRemoteException(Exception e) {
+  public static IOException handleRemoteException(Throwable e) {
     return makeIOExceptionOfException(e);
   }
 
-  private static IOException makeIOExceptionOfException(Exception e) {
+  private static IOException makeIOExceptionOfException(Throwable e) {
     Throwable t = e;
     if (e instanceof ServiceException ||
         e instanceof org.apache.hbase.thirdparty.com.google.protobuf.ServiceException) {
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 15a8c8a..a0f217c 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
@@ -41,7 +41,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ByteBufferExtendedCell;
@@ -124,6 +123,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.Service;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
@@ -345,13 +345,12 @@ public final class ProtobufUtil {
    * just {@link ServiceException}. Prefer this method to
    * {@link #getRemoteException(ServiceException)} because trying to
    * contain direct protobuf references.
-   * @param e
    */
-  public static IOException handleRemoteException(Exception e) {
+  public static IOException handleRemoteException(Throwable e) {
     return makeIOExceptionOfException(e);
   }
 
-  private static IOException makeIOExceptionOfException(Exception e) {
+  private static IOException makeIOExceptionOfException(Throwable e) {
     Throwable t = e;
     if (e instanceof ServiceException) {
       t = e.getCause();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index ba25e81..73ff37a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1940,7 +1940,7 @@ public class HRegionServer extends HasThread implements
     if (!isStopped() && !isAborted()) {
       initializeThreads();
     }
-    this.secureBulkLoadManager = new SecureBulkLoadManager(this.conf, clusterConnection);
+    this.secureBulkLoadManager = new SecureBulkLoadManager(this.conf, asyncClusterConnection);
     this.secureBulkLoadManager.start();
 
     // Health checker thread.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
index 6b55744..d54be75 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.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
@@ -27,7 +27,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.Consumer;
-
 import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -38,11 +37,12 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.regionserver.HRegion.BulkLoadListener;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
 import org.apache.hadoop.hbase.security.token.TokenUtil;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -56,7 +56,9 @@ import org.apache.hadoop.security.token.Token;
 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.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
@@ -111,9 +113,9 @@ public class SecureBulkLoadManager {
 
   private UserProvider userProvider;
   private ConcurrentHashMap<UserGroupInformation, MutableInt> ugiReferenceCounter;
-  private Connection conn;
+  private AsyncConnection conn;
 
-  SecureBulkLoadManager(Configuration conf, Connection conn) {
+  SecureBulkLoadManager(Configuration conf, AsyncConnection conn) {
     this.conf = conf;
     this.conn = conn;
   }
@@ -218,23 +220,23 @@ public class SecureBulkLoadManager {
       familyPaths.add(new Pair<>(el.getFamily().toByteArray(), el.getPath()));
     }
 
-    Token userToken = null;
+    Token<AuthenticationTokenIdentifier> userToken = null;
     if (userProvider.isHadoopSecurityEnabled()) {
-      userToken = new Token(request.getFsToken().getIdentifier().toByteArray(), request.getFsToken()
-              .getPassword().toByteArray(), new Text(request.getFsToken().getKind()), new Text(
-              request.getFsToken().getService()));
+      userToken = new Token<>(request.getFsToken().getIdentifier().toByteArray(),
+        request.getFsToken().getPassword().toByteArray(), new Text(request.getFsToken().getKind()),
+        new Text(request.getFsToken().getService()));
     }
     final String bulkToken = request.getBulkToken();
     User user = getActiveUser();
     final UserGroupInformation ugi = user.getUGI();
     if (userProvider.isHadoopSecurityEnabled()) {
       try {
-        Token tok = TokenUtil.obtainToken(conn);
+        Token<AuthenticationTokenIdentifier> tok = TokenUtil.obtainToken(conn).get();
         if (tok != null) {
           boolean b = ugi.addToken(tok);
           LOG.debug("token added " + tok + " for user " + ugi + " return=" + b);
         }
-      } catch (IOException ioe) {
+      } catch (Exception ioe) {
         LOG.warn("unable to add token", ioe);
       }
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
index c54d905..28efb84 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,27 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.security.token;
 
+import com.google.protobuf.ByteString;
+import com.google.protobuf.ServiceException;
 import java.io.IOException;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.security.PrivilegedExceptionAction;
-
-import com.google.protobuf.ByteString;
-import com.google.protobuf.ServiceException;
-
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import java.util.concurrent.CompletableFuture;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.AuthenticationService;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenRequest;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenResponse;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
@@ -45,6 +47,8 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
 /**
  * Utility methods for obtaining authentication tokens.
  */
@@ -64,12 +68,39 @@ public class TokenUtil {
 
   /**
    * Obtain and return an authentication token for the current user.
+   * @param conn The async HBase cluster connection
+   * @return the authentication token instance, wrapped by a {@link CompletableFuture}.
+   */
+  public static CompletableFuture<Token<AuthenticationTokenIdentifier>> obtainToken(
+      AsyncConnection conn) {
+    CompletableFuture<Token<AuthenticationTokenIdentifier>> future = new CompletableFuture<>();
+    if (injectedException != null) {
+      future.completeExceptionally(injectedException);
+      return future;
+    }
+    AsyncTable<?> table = conn.getTable(TableName.META_TABLE_NAME);
+    table.<AuthenticationService.Interface, GetAuthenticationTokenResponse> coprocessorService(
+      AuthenticationProtos.AuthenticationService::newStub,
+      (s, c, r) -> s.getAuthenticationToken(c,
+        AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance(), r),
+      HConstants.EMPTY_START_ROW).whenComplete((resp, error) -> {
+        if (error != null) {
+          future.completeExceptionally(ProtobufUtil.handleRemoteException(error));
+        } else {
+          future.complete(toToken(resp.getToken()));
+        }
+      });
+    return future;
+  }
+
+  /**
+   * Obtain and return an authentication token for the current user.
    * @param conn The HBase cluster connection
    * @throws IOException if a remote error or serialization problem occurs.
    * @return the authentication token instance
    */
-  public static Token<AuthenticationTokenIdentifier> obtainToken(
-      Connection conn) throws IOException {
+  public static Token<AuthenticationTokenIdentifier> obtainToken(Connection conn)
+      throws IOException {
     Table meta = null;
     try {
       injectFault();
@@ -77,9 +108,9 @@ public class TokenUtil {
       meta = conn.getTable(TableName.META_TABLE_NAME);
       CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW);
       AuthenticationProtos.AuthenticationService.BlockingInterface service =
-          AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel);
-      AuthenticationProtos.GetAuthenticationTokenResponse response = service.getAuthenticationToken(null,
-          AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance());
+        AuthenticationService.newBlockingStub(rpcChannel);
+      GetAuthenticationTokenResponse response =
+        service.getAuthenticationToken(null, GetAuthenticationTokenRequest.getDefaultInstance());
 
       return toToken(response.getToken());
     } catch (ServiceException se) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
index 32fcddb..585a3ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
@@ -18,35 +18,53 @@
 package org.apache.hadoop.hbase.security.token;
 
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.net.URL;
 import java.net.URLClassLoader;
-
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 
 @Category(SmallTests.class)
 public class TestTokenUtil {
+
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestTokenUtil.class);
+    HBaseClassTestRule.forClass(TestTokenUtil.class);
 
-  @Test
-  public void testObtainToken() throws Exception {
+  private URLClassLoader cl;
+
+  @Before
+  public void setUp() {
     URL urlPU = ProtobufUtil.class.getProtectionDomain().getCodeSource().getLocation();
     URL urlTU = TokenUtil.class.getProtectionDomain().getCodeSource().getLocation();
+    cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader());
+  }
 
-    ClassLoader cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader());
+  @After
+  public void tearDown() throws IOException {
+    Closeables.close(cl, true);
+  }
 
+  @Test
+  public void testObtainToken() throws Exception {
     Throwable injected = new com.google.protobuf.ServiceException("injected");
 
     Class<?> tokenUtil = cl.loadClass(TokenUtil.class.getCanonicalName());
@@ -55,8 +73,7 @@ public class TestTokenUtil {
     shouldInjectFault.set(null, injected);
 
     try {
-      tokenUtil.getMethod("obtainToken", Connection.class)
-          .invoke(null, new Object[] { null });
+      tokenUtil.getMethod("obtainToken", Connection.class).invoke(null, new Object[] { null });
       fail("Should have injected exception.");
     } catch (InvocationTargetException e) {
       Throwable t = e;
@@ -72,9 +89,16 @@ public class TestTokenUtil {
       }
     }
 
+    CompletableFuture<?> future = (CompletableFuture<?>) tokenUtil
+      .getMethod("obtainToken", AsyncConnection.class).invoke(null, new Object[] { null });
+    try {
+      future.get();
+      fail("Should have injected exception.");
+    } catch (ExecutionException e) {
+      assertSame(injected, e.getCause());
+    }
     Boolean loaded = (Boolean) cl.loadClass(ProtobufUtil.class.getCanonicalName())
-        .getDeclaredMethod("isClassLoaderLoaded")
-        .invoke(null);
+      .getDeclaredMethod("isClassLoaderLoaded").invoke(null);
     assertFalse("Should not have loaded DynamicClassLoader", loaded);
   }
 }


[hbase] 07/11: HBASE-21537 Rewrite ServerManager.closeRegionSilentlyAndWait to use AsyncClusterConnection

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

zhangduo pushed a commit to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit afbb50dae0d81d0e78ae0e7de2f68a6674543265
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Thu Jan 10 11:47:41 2019 +0800

    HBASE-21537 Rewrite ServerManager.closeRegionSilentlyAndWait to use AsyncClusterConnection
    
    Signed-off-by: Michael Stack <st...@apache.org>
---
 .../hadoop/hbase/master/MasterMetaBootstrap.java   |  2 +-
 .../apache/hadoop/hbase/master/ServerManager.java  | 41 +++++++++++++---------
 .../apache/hadoop/hbase/util/HBaseFsckRepair.java  | 22 +++++++-----
 3 files changed, 38 insertions(+), 27 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
index e57817e..6e38bdd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
@@ -101,7 +101,7 @@ class MasterMetaBootstrap {
           RegionState r = MetaTableLocator.getMetaRegionState(zooKeeper, replicaId);
           LOG.info("Closing excess replica of meta region " + r.getRegion());
           // send a close and wait for a max of 30 seconds
-          ServerManager.closeRegionSilentlyAndWait(master.getClusterConnection(),
+          ServerManager.closeRegionSilentlyAndWait(master.getAsyncClusterConnection(),
               r.getServerName(), r.getRegion(), 30000);
           ZKUtil.deleteNode(zooKeeper, zooKeeper.getZNodePaths().getZNodeForReplica(replicaId));
         }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index a8d5e21..f98899e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -48,13 +48,15 @@ import org.apache.hadoop.hbase.ServerMetrics;
 import org.apache.hadoop.hbase.ServerMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.YouAreDeadException;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -67,6 +69,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
@@ -660,35 +663,39 @@ public class ServerManager {
   }
 
   /**
-   * Contacts a region server and waits up to timeout ms
-   * to close the region.  This bypasses the active hmaster.
+   * Contacts a region server and waits up to timeout ms to close the region. This bypasses the
+   * active hmaster.
    */
-  public static void closeRegionSilentlyAndWait(ClusterConnection connection,
-    ServerName server, RegionInfo region, long timeout) throws IOException, InterruptedException {
-    AdminService.BlockingInterface rs = connection.getAdmin(server);
-    HBaseRpcController controller = connection.getRpcControllerFactory().newController();
+  public static void closeRegionSilentlyAndWait(AsyncClusterConnection connection,
+      ServerName server, RegionInfo region, long timeout) throws IOException, InterruptedException {
+    AsyncRegionServerAdmin admin = connection.getRegionServerAdmin(server);
     try {
-      ProtobufUtil.closeRegion(controller, rs, server, region.getRegionName());
+      FutureUtils.get(
+        admin.closeRegion(ProtobufUtil.buildCloseRegionRequest(server, region.getRegionName())));
     } catch (IOException e) {
       LOG.warn("Exception when closing region: " + region.getRegionNameAsString(), e);
     }
     long expiration = timeout + System.currentTimeMillis();
     while (System.currentTimeMillis() < expiration) {
-      controller.reset();
       try {
-        RegionInfo rsRegion =
-          ProtobufUtil.getRegionInfo(controller, rs, region.getRegionName());
-        if (rsRegion == null) return;
+        RegionInfo rsRegion = ProtobufUtil.toRegionInfo(FutureUtils
+          .get(
+            admin.getRegionInfo(RequestConverter.buildGetRegionInfoRequest(region.getRegionName())))
+          .getRegionInfo());
+        if (rsRegion == null) {
+          return;
+        }
       } catch (IOException ioe) {
-        if (ioe instanceof NotServingRegionException) // no need to retry again
+        if (ioe instanceof NotServingRegionException) {
+          // no need to retry again
           return;
-        LOG.warn("Exception when retrieving regioninfo from: "
-          + region.getRegionNameAsString(), ioe);
+        }
+        LOG.warn("Exception when retrieving regioninfo from: " + region.getRegionNameAsString(),
+          ioe);
       }
       Thread.sleep(1000);
     }
-    throw new IOException("Region " + region + " failed to close within"
-        + " timeout " + timeout);
+    throw new IOException("Region " + region + " failed to close within" + " timeout " + timeout);
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
index ec7f717..121d06c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
@@ -31,7 +31,9 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
@@ -41,6 +43,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -143,16 +146,17 @@ public class HBaseFsckRepair {
   }
 
   /**
-   * Contacts a region server and waits up to hbase.hbck.close.timeout ms
-   * (default 120s) to close the region.  This bypasses the active hmaster.
+   * Contacts a region server and waits up to hbase.hbck.close.timeout ms (default 120s) to close
+   * the region. This bypasses the active hmaster.
    */
-  @SuppressWarnings("deprecation")
-  public static void closeRegionSilentlyAndWait(Connection connection,
-      ServerName server, RegionInfo region) throws IOException, InterruptedException {
-    long timeout = connection.getConfiguration()
-      .getLong("hbase.hbck.close.timeout", 120000);
-    ServerManager.closeRegionSilentlyAndWait((ClusterConnection)connection, server,
-         region, timeout);
+  public static void closeRegionSilentlyAndWait(Connection connection, ServerName server,
+      RegionInfo region) throws IOException, InterruptedException {
+    long timeout = connection.getConfiguration().getLong("hbase.hbck.close.timeout", 120000);
+    // this is a bit ugly but it is only used in the old hbck and tests, so I think it is fine.
+    try (AsyncClusterConnection asyncConn = ClusterConnectionFactory
+      .createAsyncClusterConnection(connection.getConfiguration(), null, User.getCurrent())) {
+      ServerManager.closeRegionSilentlyAndWait(asyncConn, server, region, timeout);
+    }
   }
 
   /**


[hbase] 10/11: HBASE-21779 Reimplement BulkLoadHFilesTool to use AsyncClusterConnection

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

zhangduo pushed a commit to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit c1ead82df0fcadd9625bc92ff9d81bc535f32ffe
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Sat Feb 2 17:36:41 2019 +0800

    HBASE-21779 Reimplement BulkLoadHFilesTool to use AsyncClusterConnection
---
 .../backup/mapreduce/MapReduceRestoreJob.java      |   18 +-
 .../hadoop/hbase/backup/util/BackupUtils.java      |   17 +-
 .../hadoop/hbase/backup/util/RestoreTool.java      |    9 +-
 .../backup/TestIncrementalBackupWithBulkLoad.java  |    6 +-
 .../hadoop/hbase/client/ConnectionUtils.java       |   41 +
 .../hadoop/hbase/client/RawAsyncTableImpl.java     |   44 +-
 .../hadoop/hbase/client/SecureBulkLoadClient.java  |  150 ---
 ...estReplicationSyncUpToolWithBulkLoadedData.java |    6 +-
 .../hbase/mapreduce/IntegrationTestBulkLoad.java   |   26 +-
 .../hbase/mapreduce/IntegrationTestImportTsv.java  |    9 +-
 .../apache/hadoop/hbase/mapreduce/CopyTable.java   |   27 +-
 .../hadoop/hbase/mapreduce/HRegionPartitioner.java |    2 +-
 .../hbase/mapreduce/TestHFileOutputFormat2.java    |   28 +-
 .../hbase/client/AsyncClusterConnection.java       |   19 +
 .../hbase/client/AsyncClusterConnectionImpl.java   |   55 +
 .../hbase/client/ClusterConnectionFactory.java     |   16 +-
 .../hbase/mapreduce/LoadIncrementalHFiles.java     |   77 --
 .../mob/compactions/PartitionedMobCompactor.java   |   19 +-
 .../replication/regionserver/HFileReplicator.java  |  125 +-
 .../replication/regionserver/ReplicationSink.java  |   43 +-
 .../regionserver/WALEntrySinkFilter.java           |    8 +-
 .../hadoop/hbase/tool/BulkLoadHFilesTool.java      |  997 ++++++++++++++-
 .../hadoop/hbase/tool/LoadIncrementalHFiles.java   | 1283 --------------------
 .../org/apache/hadoop/hbase/util/HBaseFsck.java    |   11 +-
 .../hbase/client/DummyAsyncClusterConnection.java  |  155 +++
 .../hadoop/hbase/client/DummyAsyncRegistry.java    |   60 +
 .../hadoop/hbase/client/DummyAsyncTable.java       |  150 +++
 .../hbase/client/TestReplicaWithCluster.java       |   40 +-
 .../coprocessor/TestRegionObserverInterface.java   |    4 +-
 .../hbase/quotas/SpaceQuotaHelperForTests.java     |   45 +-
 .../hbase/quotas/TestLowLatencySpaceQuotas.java    |   17 +-
 .../hadoop/hbase/quotas/TestSpaceQuotas.java       |   30 +-
 .../regionserver/TestHRegionServerBulkLoad.java    |   69 +-
 .../regionserver/TestScannerWithBulkload.java      |   19 +-
 .../regionserver/TestSecureBulkLoadManager.java    |   27 +-
 .../hbase/replication/TestMasterReplication.java   |    6 +-
 .../regionserver/TestReplicationSink.java          |    6 +-
 .../regionserver/TestWALEntrySinkFilter.java       |  441 +------
 .../security/access/TestAccessController.java      |   12 +-
 ...rementalHFiles.java => TestBulkLoadHFiles.java} |  157 ++-
 .../tool/TestBulkLoadHFilesSplitRecovery.java      |  486 ++++++++
 .../TestLoadIncrementalHFilesSplitRecovery.java    |  630 ----------
 ...alHFiles.java => TestSecureBulkLoadHFiles.java} |   11 +-
 ... => TestSecureBulkLoadHFilesSplitRecovery.java} |    9 +-
 44 files changed, 2355 insertions(+), 3055 deletions(-)

diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
index 1256289..9daa282 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
@@ -17,11 +17,9 @@
  */
 package org.apache.hadoop.hbase.backup.mapreduce;
 
-import static org.apache.hadoop.hbase.backup.util.BackupUtils.failed;
 import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
 
 import java.io.IOException;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -29,7 +27,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
 import org.apache.hadoop.hbase.backup.RestoreJob;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.util.Tool;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -42,8 +40,7 @@ import org.slf4j.LoggerFactory;
  * HFiles which are aligned with a region boundaries of a table being
  * restored.
  *
- * The resulting HFiles then are loaded using HBase bulk load tool
- * {@link LoadIncrementalHFiles}
+ * The resulting HFiles then are loaded using HBase bulk load tool {@link BulkLoadHFiles}.
  */
 @InterfaceAudience.Private
 public class MapReduceRestoreJob implements RestoreJob {
@@ -88,23 +85,20 @@ public class MapReduceRestoreJob implements RestoreJob {
       };
 
       int result;
-      int loaderResult;
       try {
 
         player.setConf(getConf());
         result = player.run(playerArgs);
         if (succeeded(result)) {
           // do bulk load
-          LoadIncrementalHFiles loader = BackupUtils.createLoader(getConf());
+          BulkLoadHFiles loader = BackupUtils.createLoader(getConf());
           if (LOG.isDebugEnabled()) {
             LOG.debug("Restoring HFiles from directory " + bulkOutputPath);
           }
-          String[] args = { bulkOutputPath.toString(), newTableNames[i].getNameAsString() };
-          loaderResult = loader.run(args);
 
-          if (failed(loaderResult)) {
-            throw new IOException("Can not restore from backup directory " + dirs
-                + " (check Hadoop and HBase logs). Bulk loader return code =" + loaderResult);
+          if (loader.bulkLoad(newTableNames[i], bulkOutputPath).isEmpty()) {
+            throw new IOException("Can not restore from backup directory " + dirs +
+              " (check Hadoop and HBase logs). Bulk loader returns null");
           }
         } else {
           throw new IOException("Can not restore from backup directory " + dirs
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
index af8b954..fe2a977 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
@@ -30,7 +30,6 @@ import java.util.List;
 import java.util.Map.Entry;
 import java.util.TreeMap;
 import java.util.TreeSet;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -54,7 +53,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
@@ -722,7 +721,7 @@ public final class BackupUtils {
     return result == 0;
   }
 
-  public static LoadIncrementalHFiles createLoader(Configuration config) throws IOException {
+  public static BulkLoadHFiles createLoader(Configuration config) {
     // set configuration for restore:
     // LoadIncrementalHFile needs more time
     // <name>hbase.rpc.timeout</name> <value>600000</value>
@@ -732,15 +731,9 @@ public final class BackupUtils {
 
     // By default, it is 32 and loader will fail if # of files in any region exceed this
     // limit. Bad for snapshot restore.
-    conf.setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE);
-    conf.set(LoadIncrementalHFiles.IGNORE_UNMATCHED_CF_CONF_KEY, "yes");
-    LoadIncrementalHFiles loader;
-    try {
-      loader = new LoadIncrementalHFiles(conf);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-    return loader;
+    conf.setInt(BulkLoadHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE);
+    conf.set(BulkLoadHFiles.IGNORE_UNMATCHED_CF_CONF_KEY, "yes");
+    return BulkLoadHFiles.create(conf);
   }
 
   public static String findMostRecentBackupId(String[] backupIds) {
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
index 13b183d..92254fa 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
@@ -25,7 +25,6 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.TreeMap;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -46,7 +45,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
@@ -451,12 +450,12 @@ public class RestoreTool {
         }
       }
     }
-    return LoadIncrementalHFiles.inferBoundaries(map);
+    return BulkLoadHFilesTool.inferBoundaries(map);
   }
 
   /**
-   * Prepare the table for bulkload, most codes copied from
-   * {@link LoadIncrementalHFiles#createTable(TableName, String, Admin)}
+   * Prepare the table for bulkload, most codes copied from {@code createTable} method in
+   * {@code BulkLoadHFilesTool}.
    * @param conn connection
    * @param tableBackupPath path
    * @param tableName table name
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
index 74dd569..82f0fb7 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.tool.TestLoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.TestBulkLoadHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.junit.Assert;
@@ -92,7 +92,7 @@ public class TestIncrementalBackupWithBulkLoad extends TestBackupBase {
 
     int NB_ROWS2 = 20;
     LOG.debug("bulk loading into " + testName);
-    int actual = TestLoadIncrementalHFiles.loadHFiles(testName, table1Desc, TEST_UTIL, famName,
+    int actual = TestBulkLoadHFiles.loadHFiles(testName, table1Desc, TEST_UTIL, famName,
         qualName, false, null, new byte[][][] {
           new byte[][]{ Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
           new byte[][]{ Bytes.toBytes("ddd"), Bytes.toBytes("ooo") },
@@ -105,7 +105,7 @@ public class TestIncrementalBackupWithBulkLoad extends TestBackupBase {
     assertTrue(checkSucceeded(backupIdIncMultiple));
     // #4 bulk load again
     LOG.debug("bulk loading into " + testName);
-    int actual1 = TestLoadIncrementalHFiles.loadHFiles(testName, table1Desc, TEST_UTIL, famName,
+    int actual1 = TestBulkLoadHFiles.loadHFiles(testName, table1Desc, TEST_UTIL, famName,
       qualName, false, null,
       new byte[][][] { new byte[][] { Bytes.toBytes("ppp"), Bytes.toBytes("qqq") },
         new byte[][] { Bytes.toBytes("rrr"), Bytes.toBytes("sss") }, },
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
index fea7a1e..67a0b09 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
@@ -55,6 +56,7 @@ 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.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.io.netty.util.Timer;
 
@@ -617,4 +619,43 @@ public final class ConnectionUtils {
       return HConstants.NORMAL_QOS;
     }
   }
+
+  @FunctionalInterface
+  interface Converter<D, I, S> {
+    D convert(I info, S src) throws IOException;
+  }
+
+  @FunctionalInterface
+  interface RpcCall<RESP, REQ> {
+    void call(ClientService.Interface stub, HBaseRpcController controller, REQ req,
+        RpcCallback<RESP> done);
+  }
+
+  static <REQ, PREQ, PRESP, RESP> CompletableFuture<RESP> call(HBaseRpcController controller,
+      HRegionLocation loc, ClientService.Interface stub, REQ req,
+      Converter<PREQ, byte[], REQ> reqConvert, RpcCall<PRESP, PREQ> rpcCall,
+      Converter<RESP, HBaseRpcController, PRESP> respConverter) {
+    CompletableFuture<RESP> future = new CompletableFuture<>();
+    try {
+      rpcCall.call(stub, controller, reqConvert.convert(loc.getRegion().getRegionName(), req),
+        new RpcCallback<PRESP>() {
+
+          @Override
+          public void run(PRESP resp) {
+            if (controller.failed()) {
+              future.completeExceptionally(controller.getFailed());
+            } else {
+              try {
+                future.complete(respConverter.convert(controller, resp));
+              } catch (IOException e) {
+                future.completeExceptionally(e);
+              }
+            }
+          }
+        });
+    } catch (IOException e) {
+      future.completeExceptionally(e);
+    }
+    return future;
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
index 1925c0e..e0a49b6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.SingleRequestCallerBuilder;
+import org.apache.hadoop.hbase.client.ConnectionUtils.Converter;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
@@ -130,51 +131,14 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
     return conn.getConfiguration();
   }
 
-  @FunctionalInterface
-  private interface Converter<D, I, S> {
-    D convert(I info, S src) throws IOException;
-  }
-
-  @FunctionalInterface
-  private interface RpcCall<RESP, REQ> {
-    void call(ClientService.Interface stub, HBaseRpcController controller, REQ req,
-        RpcCallback<RESP> done);
-  }
 
-  private static <REQ, PREQ, PRESP, RESP> CompletableFuture<RESP> call(
-      HBaseRpcController controller, HRegionLocation loc, ClientService.Interface stub, REQ req,
-      Converter<PREQ, byte[], REQ> reqConvert, RpcCall<PRESP, PREQ> rpcCall,
-      Converter<RESP, HBaseRpcController, PRESP> respConverter) {
-    CompletableFuture<RESP> future = new CompletableFuture<>();
-    try {
-      rpcCall.call(stub, controller, reqConvert.convert(loc.getRegion().getRegionName(), req),
-        new RpcCallback<PRESP>() {
-
-          @Override
-          public void run(PRESP resp) {
-            if (controller.failed()) {
-              future.completeExceptionally(controller.getFailed());
-            } else {
-              try {
-                future.complete(respConverter.convert(controller, resp));
-              } catch (IOException e) {
-                future.completeExceptionally(e);
-              }
-            }
-          }
-        });
-    } catch (IOException e) {
-      future.completeExceptionally(e);
-    }
-    return future;
-  }
 
   private static <REQ, RESP> CompletableFuture<RESP> mutate(HBaseRpcController controller,
       HRegionLocation loc, ClientService.Interface stub, REQ req,
       Converter<MutateRequest, byte[], REQ> reqConvert,
       Converter<RESP, HBaseRpcController, MutateResponse> respConverter) {
-    return call(controller, loc, stub, req, reqConvert, (s, c, r, done) -> s.mutate(c, r, done),
-      respConverter);
+    return ConnectionUtils.call(controller, loc, stub, req, reqConvert,
+      (s, c, r, done) -> s.mutate(c, r, done), respConverter);
   }
 
   private static <REQ> CompletableFuture<Void> voidMutate(HBaseRpcController controller,
@@ -221,7 +185,7 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
 
   private CompletableFuture<Result> get(Get get, int replicaId) {
     return this.<Result, Get> newCaller(get, readRpcTimeoutNs)
-      .action((controller, loc, stub) -> RawAsyncTableImpl
+      .action((controller, loc, stub) -> ConnectionUtils
         .<Get, GetRequest, GetResponse, Result> call(controller, loc, stub, get,
           RequestConverter::buildGetRequest, (s, c, req, done) -> s.get(c, req, done),
           (c, resp) -> ProtobufUtil.toResult(resp.getResult(), c.cellScanner())))
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
deleted file mode 100644
index 2186271..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * 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 java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
-import org.apache.hadoop.security.token.Token;
-
-import static org.apache.hadoop.hbase.HConstants.PRIORITY_UNSET;
-
-/**
- * Client proxy for SecureBulkLoadProtocol
- */
-@InterfaceAudience.Private
-public class SecureBulkLoadClient {
-  private Table table;
-  private final RpcControllerFactory rpcControllerFactory;
-
-  public SecureBulkLoadClient(final Configuration conf, Table table) {
-    this.table = table;
-    this.rpcControllerFactory = new RpcControllerFactory(conf);
-  }
-
-  public String prepareBulkLoad(final Connection conn) throws IOException {
-    try {
-      ClientServiceCallable<String> callable = new ClientServiceCallable<String>(conn,
-          table.getName(), HConstants.EMPTY_START_ROW,
-          this.rpcControllerFactory.newController(), PRIORITY_UNSET) {
-        @Override
-        protected String rpcCall() throws Exception {
-          byte[] regionName = getLocation().getRegionInfo().getRegionName();
-          RegionSpecifier region =
-              RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
-          PrepareBulkLoadRequest request = PrepareBulkLoadRequest.newBuilder()
-              .setTableName(ProtobufUtil.toProtoTableName(table.getName()))
-              .setRegion(region).build();
-          PrepareBulkLoadResponse response = getStub().prepareBulkLoad(null, request);
-          return response.getBulkToken();
-        }
-      };
-      return RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null)
-          .<String> newCaller().callWithRetries(callable, Integer.MAX_VALUE);
-    } catch (Throwable throwable) {
-      throw new IOException(throwable);
-    }
-  }
-
-  public void cleanupBulkLoad(final Connection conn, final String bulkToken) throws IOException {
-    try {
-      ClientServiceCallable<Void> callable = new ClientServiceCallable<Void>(conn,
-          table.getName(), HConstants.EMPTY_START_ROW, this.rpcControllerFactory.newController(), PRIORITY_UNSET) {
-        @Override
-        protected Void rpcCall() throws Exception {
-          byte[] regionName = getLocation().getRegionInfo().getRegionName();
-          RegionSpecifier region = RequestConverter.buildRegionSpecifier(
-              RegionSpecifierType.REGION_NAME, regionName);
-          CleanupBulkLoadRequest request =
-              CleanupBulkLoadRequest.newBuilder().setRegion(region).setBulkToken(bulkToken).build();
-          getStub().cleanupBulkLoad(null, request);
-          return null;
-        }
-      };
-      RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null)
-          .<Void> newCaller().callWithRetries(callable, Integer.MAX_VALUE);
-    } catch (Throwable throwable) {
-      throw new IOException(throwable);
-    }
-  }
-
-  /**
-   * Securely bulk load a list of HFiles using client protocol.
-   *
-   * @param client
-   * @param familyPaths
-   * @param regionName
-   * @param assignSeqNum
-   * @param userToken
-   * @param bulkToken
-   * @return true if all are loaded
-   * @throws IOException
-   */
-  public boolean secureBulkLoadHFiles(final ClientService.BlockingInterface client,
-      final List<Pair<byte[], String>> familyPaths,
-      final byte[] regionName, boolean assignSeqNum,
-      final Token<?> userToken, final String bulkToken) throws IOException {
-    return secureBulkLoadHFiles(client, familyPaths, regionName, assignSeqNum, userToken, bulkToken,
-        false);
-  }
-
-  /**
-   * Securely bulk load a list of HFiles using client protocol.
-   *
-   * @param client
-   * @param familyPaths
-   * @param regionName
-   * @param assignSeqNum
-   * @param userToken
-   * @param bulkToken
-   * @param copyFiles
-   * @return true if all are loaded
-   * @throws IOException
-   */
-  public boolean secureBulkLoadHFiles(final ClientService.BlockingInterface client,
-      final List<Pair<byte[], String>> familyPaths,
-      final byte[] regionName, boolean assignSeqNum,
-      final Token<?> userToken, final String bulkToken, boolean copyFiles) throws IOException {
-    BulkLoadHFileRequest request =
-        RequestConverter.buildBulkLoadHFileRequest(familyPaths, regionName, assignSeqNum,
-          userToken, bulkToken, copyFiles);
-
-    try {
-      BulkLoadHFileResponse response = client.bulkLoadHFile(null, request);
-      return response.getLoaded();
-    } catch (Exception se) {
-      throw ProtobufUtil.handleRemoteException(se);
-    }
-  }
-}
diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java
index eb575c5..3e823c3 100644
--- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java
+++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.replication.regionserver.TestSourceFSConfigurationProvider;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
 import org.junit.BeforeClass;
@@ -218,9 +218,7 @@ public class TestReplicationSyncUpToolWithBulkLoadedData extends TestReplication
     }
 
     final TableName tableName = source.getName();
-    LoadIncrementalHFiles loader = new LoadIncrementalHFiles(utility1.getConfiguration());
-    String[] args = { dir.toString(), tableName.toString() };
-    loader.run(args);
+    BulkLoadHFiles.create(utility1.getConfiguration()).bulkLoad(tableName, dir);
   }
 
   private void wait(Table target, int expectedCount, String msg) throws IOException,
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
index ba2e214..4687e1b 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
@@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -60,7 +59,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.RegionSplitter;
@@ -86,6 +85,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
@@ -292,24 +292,18 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase {
 
     // Set where to place the hfiles.
     FileOutputFormat.setOutputPath(job, p);
-    try (Connection conn = ConnectionFactory.createConnection(conf);
-        Admin admin = conn.getAdmin();
-        Table table = conn.getTable(getTablename());
-        RegionLocator regionLocator = conn.getRegionLocator(getTablename())) {
-
+    try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin();
+      RegionLocator regionLocator = conn.getRegionLocator(getTablename())) {
       // Configure the partitioner and other things needed for HFileOutputFormat.
-      HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
-
+      HFileOutputFormat2.configureIncrementalLoad(job, admin.getDescriptor(getTablename()),
+        regionLocator);
       // Run the job making sure it works.
       assertEquals(true, job.waitForCompletion(true));
-
-      // Create a new loader.
-      LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
-
-      // Load the HFiles in.
-      loader.doBulkLoad(p, admin, table, regionLocator);
     }
-
+    // Create a new loader.
+    BulkLoadHFiles loader = BulkLoadHFiles.create(conf);
+    // Load the HFiles in.
+    loader.bulkLoad(getTablename(), p);
     // Delete the files.
     util.getTestFileSystem().delete(p, true);
   }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java
index ab5f2bb..c80d61c 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java
@@ -29,7 +29,6 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,7 +44,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
 import org.apache.hadoop.util.Tool;
@@ -60,7 +59,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Validate ImportTsv + LoadIncrementalHFiles on a distributed cluster.
+ * Validate ImportTsv + BulkLoadFiles on a distributed cluster.
  */
 @Category(IntegrationTests.class)
 public class IntegrationTestImportTsv extends Configured implements Tool {
@@ -141,8 +140,8 @@ public class IntegrationTestImportTsv extends Configured implements Tool {
 
     String[] args = { hfiles.toString(), tableName.getNameAsString() };
     LOG.info(format("Running LoadIncrememntalHFiles with args: %s", Arrays.asList(args)));
-    assertEquals("Loading HFiles failed.",
-      0, ToolRunner.run(new LoadIncrementalHFiles(new Configuration(getConf())), args));
+    assertEquals("Loading HFiles failed.", 0,
+      ToolRunner.run(new BulkLoadHFilesTool(getConf()), args));
 
     Table table = null;
     Scan scan = new Scan() {{
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
index b59c9e6..a443b4b 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
@@ -22,28 +22,28 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
-
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.mapreduce.Import.CellImporter;
-import org.apache.hadoop.hbase.mapreduce.Import.Importer;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.mapreduce.Import.CellImporter;
+import org.apache.hadoop.hbase.mapreduce.Import.Importer;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Tool used to copy a table to another one which can be on a different setup.
@@ -416,13 +416,12 @@ public class CopyTable extends Configured implements Tool {
     int code = 0;
     if (bulkload) {
       LOG.info("Trying to bulk load data to destination table: " + dstTableName);
-      LOG.info("command: ./bin/hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles {} {}",
+      LOG.info("command: ./bin/hbase {} {} {}", BulkLoadHFilesTool.NAME,
         this.bulkloadDir.toString(), this.dstTableName);
-      code = new LoadIncrementalHFiles(this.getConf())
-          .run(new String[] { this.bulkloadDir.toString(), this.dstTableName });
-      if (code == 0) {
-        // bulkloadDir is deleted only LoadIncrementalHFiles was successful so that one can rerun
-        // LoadIncrementalHFiles.
+      if (!BulkLoadHFiles.create(getConf()).bulkLoad(TableName.valueOf(dstTableName), bulkloadDir)
+        .isEmpty()) {
+        // bulkloadDir is deleted only BulkLoadHFiles was successful so that one can rerun
+        // BulkLoadHFiles.
         FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
         if (!fs.delete(this.bulkloadDir, true)) {
           LOG.error("Deleting folder " + bulkloadDir + " failed!");
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java
index b48ecf0..62fc06d 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.mapreduce.Partitioner;
  *
  * <p>This class is not suitable as partitioner creating hfiles
  * for incremental bulk loads as region spread will likely change between time of
- * hfile creation and load time. See {@link org.apache.hadoop.hbase.tool.LoadIncrementalHFiles}
+ * hfile creation and load time. See {@link org.apache.hadoop.hbase.tool.BulkLoadHFiles}
  * and <a href="http://hbase.apache.org/book.html#arch.bulk.load">Bulk Load</a>.</p>
  *
  * @param <KEY>  The type of the key.
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index cbf5ff8..da4c62b 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -90,7 +90,7 @@ import org.apache.hadoop.hbase.regionserver.TestHRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
@@ -700,25 +700,24 @@ public class TestHFileOutputFormat2  {
       // Perform the actual load
       for (HFileOutputFormat2.TableInfo singleTableInfo : tableInfo) {
         Path tableDir = testDir;
-        String tableNameStr = singleTableInfo.getHTableDescriptor().getNameAsString();
-        LOG.info("Running LoadIncrementalHFiles on table" + tableNameStr);
+        String tableNameStr = singleTableInfo.getTableDescriptor().getTableName().getNameAsString();
+        LOG.info("Running BulkLoadHFiles on table" + tableNameStr);
         if (writeMultipleTables) {
           tableDir = new Path(testDir, tableNameStr);
         }
         Table currentTable = allTables.get(tableNameStr);
         TableName currentTableName = currentTable.getName();
-        new LoadIncrementalHFiles(conf).doBulkLoad(tableDir, admin, currentTable, singleTableInfo
-                .getRegionLocator());
+        BulkLoadHFiles.create(conf).bulkLoad(currentTableName, tableDir);
 
         // Ensure data shows up
         int expectedRows = 0;
         if (putSortReducer) {
           // no rows should be extracted
-          assertEquals("LoadIncrementalHFiles should put expected data in table", expectedRows,
+          assertEquals("BulkLoadHFiles should put expected data in table", expectedRows,
                   util.countRows(currentTable));
         } else {
           expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
-          assertEquals("LoadIncrementalHFiles should put expected data in table", expectedRows,
+          assertEquals("BulkLoadHFiles should put expected data in table", expectedRows,
                   util.countRows(currentTable));
           Scan scan = new Scan();
           ResultScanner results = currentTable.getScanner(scan);
@@ -1248,14 +1247,14 @@ public class TestHFileOutputFormat2  {
       for (int i = 0; i < 2; i++) {
         Path testDir = util.getDataTestDirOnTestFS("testExcludeAllFromMinorCompaction_" + i);
         runIncrementalPELoad(conf, Arrays.asList(new HFileOutputFormat2.TableInfo(table
-                .getTableDescriptor(), conn.getRegionLocator(TABLE_NAMES[0]))), testDir, false);
+                .getDescriptor(), conn.getRegionLocator(TABLE_NAMES[0]))), testDir, false);
         // Perform the actual load
-        new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, locator);
+        BulkLoadHFiles.create(conf).bulkLoad(table.getName(), testDir);
       }
 
       // Ensure data shows up
       int expectedRows = 2 * NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
-      assertEquals("LoadIncrementalHFiles should put expected data in table",
+      assertEquals("BulkLoadHFiles should put expected data in table",
           expectedRows, util.countRows(table));
 
       // should have a second StoreFile now
@@ -1340,15 +1339,16 @@ public class TestHFileOutputFormat2  {
           true);
 
       RegionLocator regionLocator = conn.getRegionLocator(TABLE_NAMES[0]);
-      runIncrementalPELoad(conf, Arrays.asList(new HFileOutputFormat2.TableInfo(table
-                      .getTableDescriptor(), regionLocator)), testDir, false);
+      runIncrementalPELoad(conf,
+        Arrays.asList(new HFileOutputFormat2.TableInfo(table.getDescriptor(), regionLocator)),
+        testDir, false);
 
       // Perform the actual load
-      new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, regionLocator);
+      BulkLoadHFiles.create(conf).bulkLoad(table.getName(), testDir);
 
       // Ensure data shows up
       int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
-      assertEquals("LoadIncrementalHFiles should put expected data in table",
+      assertEquals("BulkLoadHFiles should put expected data in table",
           expectedRows + 1, util.countRows(table));
 
       // should have a second StoreFile now
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
index 0ad77ba..c3f8f8b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
@@ -23,7 +23,9 @@ import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.security.token.Token;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
@@ -66,4 +68,21 @@ public interface AsyncClusterConnection extends AsyncConnection {
    */
   CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
       boolean reload);
+
+  /**
+   * Return the token for this bulk load.
+   */
+  CompletableFuture<String> prepareBulkLoad(TableName tableName);
+
+  /**
+   * Securely bulk load a list of HFiles.
+   * @param row used to locate the region
+   */
+  CompletableFuture<Boolean> bulkLoad(TableName tableName, List<Pair<byte[], String>> familyPaths,
+      byte[] row, boolean assignSeqNum, Token<?> userToken, String bulkToken, boolean copyFiles);
+
+  /**
+   * Clean up after finishing bulk load, no matter success or not.
+   */
+  CompletableFuture<Void> cleanupBulkLoad(TableName tableName, String bulkToken);
 }
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 d61f01f..328b959 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
@@ -21,15 +21,28 @@ import java.net.SocketAddress;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.security.token.Token;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 
 /**
  * The implementation of AsyncClusterConnection.
@@ -77,4 +90,46 @@ class AsyncClusterConnectionImpl extends AsyncConnectionImpl implements AsyncClu
       boolean reload) {
     return getLocator().getRegionLocations(tableName, row, RegionLocateType.CURRENT, reload, -1L);
   }
+
+  @Override
+  public CompletableFuture<String> prepareBulkLoad(TableName tableName) {
+    return callerFactory.<String> single().table(tableName).row(HConstants.EMPTY_START_ROW)
+      .action((controller, loc, stub) -> ConnectionUtils
+        .<TableName, PrepareBulkLoadRequest, PrepareBulkLoadResponse, String> call(controller, loc,
+          stub, tableName, (rn, tn) -> {
+            RegionSpecifier region =
+              RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, rn);
+            return PrepareBulkLoadRequest.newBuilder()
+              .setTableName(ProtobufUtil.toProtoTableName(tn)).setRegion(region).build();
+          }, (s, c, req, done) -> s.prepareBulkLoad(c, req, done),
+          (c, resp) -> resp.getBulkToken()))
+      .call();
+  }
+
+  @Override
+  public CompletableFuture<Boolean> bulkLoad(TableName tableName,
+      List<Pair<byte[], String>> familyPaths, byte[] row, boolean assignSeqNum, Token<?> userToken,
+      String bulkToken, boolean copyFiles) {
+    return callerFactory.<Boolean> single().table(tableName).row(row)
+      .action((controller, loc, stub) -> ConnectionUtils
+        .<Void, BulkLoadHFileRequest, BulkLoadHFileResponse, Boolean> call(controller, loc, stub,
+          null,
+          (rn, nil) -> RequestConverter.buildBulkLoadHFileRequest(familyPaths, rn, assignSeqNum,
+            userToken, bulkToken, copyFiles),
+          (s, c, req, done) -> s.bulkLoadHFile(c, req, done), (c, resp) -> resp.getLoaded()))
+      .call();
+  }
+
+  @Override
+  public CompletableFuture<Void> cleanupBulkLoad(TableName tableName, String bulkToken) {
+    return callerFactory.<Void> single().table(tableName).row(HConstants.EMPTY_START_ROW)
+      .action((controller, loc, stub) -> ConnectionUtils
+        .<String, CleanupBulkLoadRequest, CleanupBulkLoadResponse, Void> call(controller, loc, stub,
+          bulkToken, (rn, bt) -> {
+            RegionSpecifier region =
+              RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, rn);
+            return CleanupBulkLoadRequest.newBuilder().setRegion(region).setBulkToken(bt).build();
+          }, (s, c, req, done) -> s.cleanupBulkLoad(c, req, done), (c, resp) -> null))
+      .call();
+  }
 }
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 2670420..46c0f5a 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
@@ -19,9 +19,11 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 import java.net.SocketAddress;
+import java.security.PrivilegedExceptionAction;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.FutureUtils;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -30,6 +32,9 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public final class ClusterConnectionFactory {
 
+  public static final String HBASE_SERVER_CLUSTER_CONNECTION_IMPL =
+    "hbase.server.cluster.connection.impl";
+
   private ClusterConnectionFactory() {
   }
 
@@ -46,6 +51,15 @@ public final class ClusterConnectionFactory {
       SocketAddress localAddress, User user) throws IOException {
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf);
     String clusterId = FutureUtils.get(registry.getClusterId());
-    return new AsyncClusterConnectionImpl(conf, registry, clusterId, localAddress, user);
+    Class<? extends AsyncClusterConnection> clazz =
+      conf.getClass(HBASE_SERVER_CLUSTER_CONNECTION_IMPL, AsyncClusterConnectionImpl.class,
+        AsyncClusterConnection.class);
+    try {
+      return user
+        .runAs((PrivilegedExceptionAction<? extends AsyncClusterConnection>) () -> ReflectionUtils
+          .newInstance(clazz, conf, registry, clusterId, localAddress, user));
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
deleted file mode 100644
index 6f5412f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- *
- * 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.mapreduce;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Tool to load the output of HFileOutputFormat into an existing table.
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
- *             {@link org.apache.hadoop.hbase.tool.LoadIncrementalHFiles} instead.
- */
-@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Temporary glue. To be removed")
-@Deprecated
-@InterfaceAudience.Public
-public class LoadIncrementalHFiles extends org.apache.hadoop.hbase.tool.LoadIncrementalHFiles {
-
-  /**
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
-   *             {@link org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem} instead.
-   */
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-      justification = "Temporary glue. To be removed")
-  @Deprecated
-  @InterfaceAudience.Public
-  public static class LoadQueueItem
-      extends org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem {
-
-    public LoadQueueItem(byte[] family, Path hfilePath) {
-      super(family, hfilePath);
-    }
-  }
-
-  public LoadIncrementalHFiles(Configuration conf) {
-    super(conf);
-  }
-
-  public Map<LoadQueueItem, ByteBuffer> run(String dirPath, Map<byte[], List<Path>> map,
-      TableName tableName) throws IOException {
-    Map<org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem, ByteBuffer> originRet;
-    if (dirPath != null) {
-      originRet = run(dirPath, tableName);
-    } else {
-      originRet = run(map, tableName);
-    }
-    Map<LoadQueueItem, ByteBuffer> ret = new HashMap<>();
-    originRet.forEach((k, v) -> {
-      ret.put(new LoadQueueItem(k.getFamily(), k.getFilePath()), v);
-    });
-    return ret;
-  }
-}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
index 9f1ab96..a5823ec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
@@ -40,7 +40,6 @@ import java.util.TreeMap;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -82,8 +81,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
@@ -91,6 +89,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 /**
  * An implementation of {@link MobCompactor} that compacts the mob files in partitions.
  */
@@ -675,7 +675,7 @@ public class PartitionedMobCompactor extends MobCompactor {
         cleanupTmpMobFile = false;
         cleanupCommittedMobFile = true;
         // bulkload the ref file
-        bulkloadRefFile(connection, table, bulkloadPathOfPartition, filePath.getName());
+        bulkloadRefFile(table.getName(), bulkloadPathOfPartition, filePath.getName());
         cleanupCommittedMobFile = false;
         newFiles.add(new Path(mobFamilyDir, filePath.getName()));
       }
@@ -818,21 +818,16 @@ public class PartitionedMobCompactor extends MobCompactor {
 
   /**
    * Bulkloads the current file.
-   *
-   * @param connection to use to get admin/RegionLocator
-   * @param table The current table.
+   * @param tableName The table to load into.
    * @param bulkloadDirectory The path of bulkload directory.
    * @param fileName The current file name.
    * @throws IOException if IO failure is encountered
    */
-  private void bulkloadRefFile(Connection connection, Table table, Path bulkloadDirectory,
-      String fileName)
+  private void bulkloadRefFile(TableName tableName, Path bulkloadDirectory, String fileName)
       throws IOException {
     // bulkload the ref file
     try {
-      LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
-      bulkload.doBulkLoad(bulkloadDirectory, connection.getAdmin(), table,
-          connection.getRegionLocator(table.getName()));
+      BulkLoadHFiles.create(conf).bulkLoad(tableName, bulkloadDirectory);
     } catch (Exception e) {
       throw new IOException(e);
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
index 1f44817..6204ea5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
@@ -1,17 +1,22 @@
-/*
- * 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.
+/**
+ * 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.replication.regionserver;
 
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -30,33 +35,32 @@ import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles.LoadQueueItem;
+import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
  * It is used for replicating HFile entries. It will first copy parallely all the hfiles to a local
- * staging directory and then it will use ({@link LoadIncrementalHFiles} to prepare a collection of
+ * staging directory and then it will use ({@link BulkLoadHFiles} to prepare a collection of
  * {@link LoadQueueItem} which will finally be loaded(replicated) into the table of this cluster.
  */
 @InterfaceAudience.Private
@@ -82,7 +86,7 @@ public class HFileReplicator {
   private FsDelegationToken fsDelegationToken;
   private UserProvider userProvider;
   private Configuration conf;
-  private Connection connection;
+  private AsyncClusterConnection connection;
   private Path hbaseStagingDir;
   private ThreadPoolExecutor exec;
   private int maxCopyThreads;
@@ -91,7 +95,7 @@ public class HFileReplicator {
   public HFileReplicator(Configuration sourceClusterConf,
       String sourceBaseNamespaceDirPath, String sourceHFileArchiveDirPath,
       Map<String, List<Pair<byte[], List<String>>>> tableQueueMap, Configuration conf,
-      Connection connection) throws IOException {
+      AsyncClusterConnection connection) throws IOException {
     this.sourceClusterConf = sourceClusterConf;
     this.sourceBaseNamespaceDirPath = sourceBaseNamespaceDirPath;
     this.sourceHFileArchiveDirPath = sourceHFileArchiveDirPath;
@@ -128,96 +132,61 @@ public class HFileReplicator {
       String tableNameString = tableStagingDir.getKey();
       Path stagingDir = tableStagingDir.getValue();
 
-      LoadIncrementalHFiles loadHFiles = null;
-      try {
-        loadHFiles = new LoadIncrementalHFiles(conf);
-      } catch (Exception e) {
-        LOG.error("Failed to initialize LoadIncrementalHFiles for replicating bulk loaded"
-            + " data.", e);
-        throw new IOException(e);
-      }
-      Configuration newConf = HBaseConfiguration.create(conf);
-      newConf.set(LoadIncrementalHFiles.CREATE_TABLE_CONF_KEY, "no");
-      loadHFiles.setConf(newConf);
-
       TableName tableName = TableName.valueOf(tableNameString);
-      Table table = this.connection.getTable(tableName);
 
       // Prepare collection of queue of hfiles to be loaded(replicated)
       Deque<LoadQueueItem> queue = new LinkedList<>();
-      loadHFiles.prepareHFileQueue(stagingDir, table, queue, false);
+      BulkLoadHFilesTool.prepareHFileQueue(conf, connection, tableName, stagingDir, queue, false,
+        false);
 
       if (queue.isEmpty()) {
         LOG.warn("Replication process did not find any files to replicate in directory "
             + stagingDir.toUri());
         return null;
       }
-
-      try (RegionLocator locator = connection.getRegionLocator(tableName)) {
-
-        fsDelegationToken.acquireDelegationToken(sinkFs);
-
-        // Set the staging directory which will be used by LoadIncrementalHFiles for loading the
-        // data
-        loadHFiles.setBulkToken(stagingDir.toString());
-
-        doBulkLoad(loadHFiles, table, queue, locator, maxRetries);
+      fsDelegationToken.acquireDelegationToken(sinkFs);
+      try {
+        doBulkLoad(conf, tableName, stagingDir, queue, maxRetries);
       } finally {
-        cleanup(stagingDir.toString(), table);
+        cleanup(stagingDir);
       }
     }
     return null;
   }
 
-  private void doBulkLoad(LoadIncrementalHFiles loadHFiles, Table table,
-      Deque<LoadQueueItem> queue, RegionLocator locator, int maxRetries) throws IOException {
-    int count = 0;
-    Pair<byte[][], byte[][]> startEndKeys;
-    while (!queue.isEmpty()) {
-      // need to reload split keys each iteration.
-      startEndKeys = locator.getStartEndKeys();
+  private void doBulkLoad(Configuration conf, TableName tableName, Path stagingDir,
+      Deque<LoadQueueItem> queue, int maxRetries) throws IOException {
+    BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf);
+    // Set the staging directory which will be used by BulkLoadHFilesTool for loading the data
+    loader.setBulkToken(stagingDir.toString());
+    for (int count = 0; !queue.isEmpty(); count++) {
       if (count != 0) {
-        LOG.warn("Error occurred while replicating HFiles, retry attempt " + count + " with "
-            + queue.size() + " files still remaining to replicate.");
+        LOG.warn("Error occurred while replicating HFiles, retry attempt " + count + " with " +
+          queue.size() + " files still remaining to replicate.");
       }
 
       if (maxRetries != 0 && count >= maxRetries) {
-        throw new IOException("Retry attempted " + count
-            + " times without completing, bailing out.");
+        throw new IOException(
+          "Retry attempted " + count + " times without completing, bailing out.");
       }
-      count++;
 
       // Try bulk load
-      loadHFiles.loadHFileQueue(table, connection, queue, startEndKeys);
+      loader.loadHFileQueue(connection, tableName, queue, false);
     }
   }
 
-  private void cleanup(String stagingDir, Table table) {
+  private void cleanup(Path stagingDir) {
     // Release the file system delegation token
     fsDelegationToken.releaseDelegationToken();
     // Delete the staging directory
     if (stagingDir != null) {
       try {
-        sinkFs.delete(new Path(stagingDir), true);
+        sinkFs.delete(stagingDir, true);
       } catch (IOException e) {
         LOG.warn("Failed to delete the staging directory " + stagingDir, e);
       }
     }
     // Do not close the file system
-
-    /*
-     * if (sinkFs != null) { try { sinkFs.close(); } catch (IOException e) { LOG.warn(
-     * "Failed to close the file system"); } }
-     */
-
-    // Close the table
-    if (table != null) {
-      try {
-        table.close();
-      } catch (IOException e) {
-        LOG.warn("Failed to close the table.", e);
-      }
-    }
   }
 
   private Map<String, Path> copyHFilesToStagingDir() throws IOException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
index 3cd928a..e30e637 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
-import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -27,8 +26,9 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.TreeMap;
 import java.util.UUID;
+import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicLong;
-
+import java.util.stream.Collectors;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -40,16 +40,18 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
+import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -83,7 +85,7 @@ public class ReplicationSink {
   private final Configuration conf;
   // Volatile because of note in here -- look for double-checked locking:
   // http://www.oracle.com/technetwork/articles/javase/bloch-effective-08-qa-140880.html
-  private volatile Connection sharedConn;
+  private volatile AsyncClusterConnection sharedConn;
   private final MetricsSink metrics;
   private final AtomicLong totalReplicatedEdits = new AtomicLong();
   private final Object sharedConnLock = new Object();
@@ -390,37 +392,34 @@ public class ReplicationSink {
    * Do the changes and handle the pool
    * @param tableName table to insert into
    * @param allRows list of actions
-   * @throws IOException
    */
   private void batch(TableName tableName, Collection<List<Row>> allRows) throws IOException {
     if (allRows.isEmpty()) {
       return;
     }
-    Connection connection = getConnection();
-    try (Table table = connection.getTable(tableName)) {
-      for (List<Row> rows : allRows) {
-        table.batch(rows, null);
-      }
-    } catch (RetriesExhaustedWithDetailsException rewde) {
-      for (Throwable ex : rewde.getCauses()) {
-        if (ex instanceof TableNotFoundException) {
+    AsyncTable<?> table = getConnection().getTable(tableName);
+    List<Future<?>> futures = allRows.stream().map(table::batchAll).collect(Collectors.toList());
+    for (Future<?> future : futures) {
+      try {
+        FutureUtils.get(future);
+      } catch (RetriesExhaustedException e) {
+        if (e.getCause() instanceof TableNotFoundException) {
           throw new TableNotFoundException("'" + tableName + "'");
         }
+        throw e;
       }
-      throw rewde;
-    } catch (InterruptedException ix) {
-      throw (InterruptedIOException) new InterruptedIOException().initCause(ix);
     }
   }
 
-  private Connection getConnection() throws IOException {
+  private AsyncClusterConnection getConnection() throws IOException {
     // See https://en.wikipedia.org/wiki/Double-checked_locking
-    Connection connection = sharedConn;
+    AsyncClusterConnection connection = sharedConn;
     if (connection == null) {
       synchronized (sharedConnLock) {
         connection = sharedConn;
         if (connection == null) {
-          connection = ConnectionFactory.createConnection(conf);
+          connection = ClusterConnectionFactory.createAsyncClusterConnection(conf, null,
+            UserProvider.instantiate(conf).getCurrent());
           sharedConn = connection;
         }
       }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntrySinkFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntrySinkFilter.java
index f0b13e1..6f6ae1f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntrySinkFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntrySinkFilter.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,9 +18,10 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 
 /**
  * Implementations are installed on a Replication Sink called from inside
@@ -36,6 +37,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  * source-side.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
+@InterfaceStability.Evolving
 public interface WALEntrySinkFilter {
   /**
    * Name of configuration to set with name of implementing WALEntrySinkFilter class.
@@ -46,7 +48,7 @@ public interface WALEntrySinkFilter {
    * Called after Construction.
    * Use passed Connection to keep any context the filter might need.
    */
-  void init(Connection connection);
+  void init(AsyncConnection conn);
 
   /**
    * @param table Table edit is destined for.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java
index 795bd66..dfdb27a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java
@@ -17,48 +17,1027 @@
  */
 package org.apache.hadoop.hbase.tool;
 
+import static java.lang.String.format;
+
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+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.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
+import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.AsyncAdmin;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.io.HalfStoreFileReader;
+import org.apache.hadoop.hbase.io.Reference;
+import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileContext;
+import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.FsDelegationToken;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSVisitor;
+import org.apache.hadoop.hbase.util.FutureUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 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.collect.HashMultimap;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
+import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
+import org.apache.hbase.thirdparty.com.google.common.collect.Multimaps;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
  * The implementation for {@link BulkLoadHFiles}, and also can be executed from command line as a
  * tool.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-public class BulkLoadHFilesTool extends LoadIncrementalHFiles implements BulkLoadHFiles {
+public class BulkLoadHFilesTool extends Configured implements BulkLoadHFiles, Tool {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BulkLoadHFilesTool.class);
 
   public static final String NAME = "completebulkload";
 
+  // We use a '.' prefix which is ignored when walking directory trees
+  // above. It is invalid family name.
+  static final String TMP_DIR = ".tmp";
+
+  private final int maxFilesPerRegionPerFamily;
+  private final boolean assignSeqIds;
+
+  // Source delegation token
+  private final FsDelegationToken fsDelegationToken;
+  private final UserProvider userProvider;
+  private final int nrThreads;
+  private final AtomicInteger numRetries = new AtomicInteger(0);
+  private String bulkToken;
+
   public BulkLoadHFilesTool(Configuration conf) {
-    super(conf);
+    // make a copy, just to be sure we're not overriding someone else's config
+    super(new Configuration(conf));
+    // disable blockcache for tool invocation, see HBASE-10500
+    conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0);
+    userProvider = UserProvider.instantiate(conf);
+    fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
+    assignSeqIds = conf.getBoolean(ASSIGN_SEQ_IDS, true);
+    maxFilesPerRegionPerFamily = conf.getInt(MAX_FILES_PER_REGION_PER_FAMILY, 32);
+    nrThreads =
+      conf.getInt("hbase.loadincremental.threads.max", Runtime.getRuntime().availableProcessors());
+  }
+
+  // Initialize a thread pool
+  private ExecutorService createExecutorService() {
+    ThreadPoolExecutor pool = new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS,
+      new LinkedBlockingQueue<>(),
+      new ThreadFactoryBuilder().setNameFormat("BulkLoadHFilesTool-%1$d").build());
+    pool.allowCoreThreadTimeOut(true);
+    return pool;
+  }
+
+  private boolean isCreateTable() {
+    return "yes".equalsIgnoreCase(getConf().get(CREATE_TABLE_CONF_KEY, "yes"));
+  }
+
+  private boolean isSilence() {
+    return "yes".equalsIgnoreCase(getConf().get(IGNORE_UNMATCHED_CF_CONF_KEY, ""));
+  }
+
+  private boolean isAlwaysCopyFiles() {
+    return getConf().getBoolean(ALWAYS_COPY_FILES, false);
+  }
+
+  private static boolean shouldCopyHFileMetaKey(byte[] key) {
+    // skip encoding to keep hfile meta consistent with data block info, see HBASE-15085
+    if (Bytes.equals(key, HFileDataBlockEncoder.DATA_BLOCK_ENCODING)) {
+      return false;
+    }
+
+    return !HFile.isReservedFileInfoKey(key);
+  }
+
+  /**
+   * Checks whether there is any invalid family name in HFiles to be bulk loaded.
+   */
+  private static void validateFamiliesInHFiles(TableDescriptor tableDesc,
+      Deque<LoadQueueItem> queue, boolean silence) throws IOException {
+    Set<String> familyNames = Arrays.asList(tableDesc.getColumnFamilies()).stream()
+      .map(f -> f.getNameAsString()).collect(Collectors.toSet());
+    List<String> unmatchedFamilies = queue.stream().map(item -> Bytes.toString(item.getFamily()))
+      .filter(fn -> !familyNames.contains(fn)).distinct().collect(Collectors.toList());
+    if (unmatchedFamilies.size() > 0) {
+      String msg =
+        "Unmatched family names found: unmatched family names in HFiles to be bulkloaded: " +
+          unmatchedFamilies + "; valid family names of table " + tableDesc.getTableName() +
+          " are: " + familyNames;
+      LOG.error(msg);
+      if (!silence) {
+        throw new IOException(msg);
+      }
+    }
+  }
+
+  /**
+   * Populate the Queue with given HFiles
+   */
+  private static void populateLoadQueue(Deque<LoadQueueItem> ret, Map<byte[], List<Path>> map) {
+    map.forEach((k, v) -> v.stream().map(p -> new LoadQueueItem(k, p)).forEachOrdered(ret::add));
+  }
+
+  private interface BulkHFileVisitor<TFamily> {
+
+    TFamily bulkFamily(byte[] familyName) throws IOException;
+
+    void bulkHFile(TFamily family, FileStatus hfileStatus) throws IOException;
+  }
+
+  /**
+   * Iterate over the bulkDir hfiles. Skip reference, HFileLink, files starting with "_". Check and
+   * skip non-valid hfiles by default, or skip this validation by setting
+   * 'hbase.loadincremental.validate.hfile' to false.
+   */
+  private static <TFamily> void visitBulkHFiles(FileSystem fs, Path bulkDir,
+      BulkHFileVisitor<TFamily> visitor, boolean validateHFile) throws IOException {
+    FileStatus[] familyDirStatuses = fs.listStatus(bulkDir);
+    for (FileStatus familyStat : familyDirStatuses) {
+      if (!familyStat.isDirectory()) {
+        LOG.warn("Skipping non-directory " + familyStat.getPath());
+        continue;
+      }
+      Path familyDir = familyStat.getPath();
+      byte[] familyName = Bytes.toBytes(familyDir.getName());
+      // Skip invalid family
+      try {
+        ColumnFamilyDescriptorBuilder.isLegalColumnFamilyName(familyName);
+      } catch (IllegalArgumentException e) {
+        LOG.warn("Skipping invalid " + familyStat.getPath());
+        continue;
+      }
+      TFamily family = visitor.bulkFamily(familyName);
+
+      FileStatus[] hfileStatuses = fs.listStatus(familyDir);
+      for (FileStatus hfileStatus : hfileStatuses) {
+        if (!fs.isFile(hfileStatus.getPath())) {
+          LOG.warn("Skipping non-file " + hfileStatus);
+          continue;
+        }
+
+        Path hfile = hfileStatus.getPath();
+        // Skip "_", reference, HFileLink
+        String fileName = hfile.getName();
+        if (fileName.startsWith("_")) {
+          continue;
+        }
+        if (StoreFileInfo.isReference(fileName)) {
+          LOG.warn("Skipping reference " + fileName);
+          continue;
+        }
+        if (HFileLink.isHFileLink(fileName)) {
+          LOG.warn("Skipping HFileLink " + fileName);
+          continue;
+        }
+
+        // Validate HFile Format if needed
+        if (validateHFile) {
+          try {
+            if (!HFile.isHFileFormat(fs, hfile)) {
+              LOG.warn("the file " + hfile + " doesn't seems to be an hfile. skipping");
+              continue;
+            }
+          } catch (FileNotFoundException e) {
+            LOG.warn("the file " + hfile + " was removed");
+            continue;
+          }
+        }
+
+        visitor.bulkHFile(family, hfileStatus);
+      }
+    }
+  }
+
+  /**
+   * Walk the given directory for all HFiles, and return a Queue containing all such files.
+   */
+  private static void discoverLoadQueue(Configuration conf, Deque<LoadQueueItem> ret, Path hfofDir,
+      boolean validateHFile) throws IOException {
+    visitBulkHFiles(hfofDir.getFileSystem(conf), hfofDir, new BulkHFileVisitor<byte[]>() {
+      @Override
+      public byte[] bulkFamily(final byte[] familyName) {
+        return familyName;
+      }
+
+      @Override
+      public void bulkHFile(final byte[] family, final FileStatus hfile) throws IOException {
+        long length = hfile.getLen();
+        if (length > conf.getLong(HConstants.HREGION_MAX_FILESIZE,
+          HConstants.DEFAULT_MAX_FILE_SIZE)) {
+          LOG.warn("Trying to bulk load hfile " + hfile.getPath() + " with size: " + length +
+            " bytes can be problematic as it may lead to oversplitting.");
+        }
+        ret.add(new LoadQueueItem(family, hfile.getPath()));
+      }
+    }, validateHFile);
+  }
+
+  /**
+   * Prepare a collection of {@code LoadQueueItem} from list of source hfiles contained in the
+   * passed directory and validates whether the prepared queue has all the valid table column
+   * families in it.
+   * @param map map of family to List of hfiles
+   * @param tableName table to which hfiles should be loaded
+   * @param queue queue which needs to be loaded into the table
+   * @param silence true to ignore unmatched column families
+   * @throws IOException If any I/O or network error occurred
+   */
+  public static void prepareHFileQueue(AsyncClusterConnection conn, TableName tableName,
+      Map<byte[], List<Path>> map, Deque<LoadQueueItem> queue, boolean silence) throws IOException {
+    populateLoadQueue(queue, map);
+    validateFamiliesInHFiles(FutureUtils.get(conn.getAdmin().getDescriptor(tableName)), queue,
+      silence);
+  }
+
+  /**
+   * Prepare a collection of {@code LoadQueueItem} from list of source hfiles contained in the
+   * passed directory and validates whether the prepared queue has all the valid table column
+   * families in it.
+   * @param hfilesDir directory containing list of hfiles to be loaded into the table
+   * @param queue queue which needs to be loaded into the table
+   * @param validateHFile if true hfiles will be validated for its format
+   * @param silence true to ignore unmatched column families
+   * @throws IOException If any I/O or network error occurred
+   */
+  public static void prepareHFileQueue(Configuration conf, AsyncClusterConnection conn,
+      TableName tableName, Path hfilesDir, Deque<LoadQueueItem> queue, boolean validateHFile,
+      boolean silence) throws IOException {
+    discoverLoadQueue(conf, queue, hfilesDir, validateHFile);
+    validateFamiliesInHFiles(FutureUtils.get(conn.getAdmin().getDescriptor(tableName)), queue,
+      silence);
+  }
+
+  /**
+   * Used by the replication sink to load the hfiles from the source cluster. It does the following,
+   * <ol>
+   * <li>{@link #groupOrSplitPhase(AsyncClusterConnection, TableName, ExecutorService, Deque, List)}
+   * </li>
+   * <li>{@link #bulkLoadPhase(AsyncClusterConnection, TableName, Deque, Multimap, boolean, Map)}
+   * </li>
+   * </ol>
+   * @param conn Connection to use
+   * @param tableName Table to which these hfiles should be loaded to
+   * @param queue {@code LoadQueueItem} has hfiles yet to be loaded
+   */
+  public void loadHFileQueue(AsyncClusterConnection conn, TableName tableName,
+      Deque<LoadQueueItem> queue, boolean copyFiles) throws IOException {
+    ExecutorService pool = createExecutorService();
+    try {
+      Multimap<ByteBuffer, LoadQueueItem> regionGroups = groupOrSplitPhase(conn, tableName, pool,
+        queue, FutureUtils.get(conn.getRegionLocator(tableName).getStartEndKeys())).getFirst();
+      bulkLoadPhase(conn, tableName, queue, regionGroups, copyFiles, null);
+    } finally {
+      pool.shutdown();
+    }
+  }
+
+  /**
+   * This takes the LQI's grouped by likely regions and attempts to bulk load them. Any failures are
+   * re-queued for another pass with the groupOrSplitPhase.
+   * <p/>
+   * protected for testing.
+   */
+  @VisibleForTesting
+  protected void bulkLoadPhase(AsyncClusterConnection conn, TableName tableName,
+      Deque<LoadQueueItem> queue, Multimap<ByteBuffer, LoadQueueItem> regionGroups,
+      boolean copyFiles, Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
+    // atomically bulk load the groups.
+    List<Future<Collection<LoadQueueItem>>> loadingFutures = new ArrayList<>();
+    for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> entry : regionGroups.asMap()
+      .entrySet()) {
+      byte[] first = entry.getKey().array();
+      final Collection<LoadQueueItem> lqis = entry.getValue();
+      List<Pair<byte[], String>> familyPaths =
+        lqis.stream().map(lqi -> Pair.newPair(lqi.getFamily(), lqi.getFilePath().toString()))
+          .collect(Collectors.toList());
+      CompletableFuture<Collection<LoadQueueItem>> future = new CompletableFuture<>();
+      FutureUtils.addListener(conn.bulkLoad(tableName, familyPaths, first, assignSeqIds,
+        fsDelegationToken.getUserToken(), bulkToken, copyFiles), (loaded, error) -> {
+          if (error != null) {
+            LOG.error("Encountered unrecoverable error from region server", error);
+            if (getConf().getBoolean(RETRY_ON_IO_EXCEPTION, false) &&
+              numRetries.get() < getConf().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+                HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER)) {
+              LOG.warn("Will attempt to retry loading failed HFiles. Retry #" +
+                numRetries.incrementAndGet());
+              // return lqi's to retry
+              future.complete(lqis);
+            } else {
+              LOG.error(RETRY_ON_IO_EXCEPTION +
+                " is disabled or we have reached retry limit. Unable to recover");
+              future.completeExceptionally(error);
+            }
+          } else {
+            if (loaded) {
+              future.complete(Collections.emptyList());
+            } else {
+              LOG.warn("Attempt to bulk load region containing " + Bytes.toStringBinary(first) +
+                " into table " + tableName + " with files " + lqis +
+                " failed.  This is recoverable and they will be retried.");
+              // return lqi's to retry
+              future.complete(lqis);
+            }
+          }
+        });
+      loadingFutures.add(future);
+      if (item2RegionMap != null) {
+        for (LoadQueueItem lqi : lqis) {
+          item2RegionMap.put(lqi, entry.getKey());
+        }
+      }
+    }
+
+    // get all the results.
+    for (Future<Collection<LoadQueueItem>> future : loadingFutures) {
+      try {
+        Collection<LoadQueueItem> toRetry = future.get();
+
+        for (LoadQueueItem lqi : toRetry) {
+          item2RegionMap.remove(lqi);
+        }
+        // LQIs that are requeued to be regrouped.
+        queue.addAll(toRetry);
+      } catch (ExecutionException e1) {
+        Throwable t = e1.getCause();
+        if (t instanceof IOException) {
+          // At this point something unrecoverable has happened.
+          // TODO Implement bulk load recovery
+          throw new IOException("BulkLoad encountered an unrecoverable problem", t);
+        }
+        LOG.error("Unexpected execution exception during bulk load", e1);
+        throw new IllegalStateException(t);
+      } catch (InterruptedException e1) {
+        LOG.error("Unexpected interrupted exception during bulk load", e1);
+        throw (InterruptedIOException) new InterruptedIOException().initCause(e1);
+      }
+    }
+  }
+
+  private boolean checkHFilesCountPerRegionPerFamily(
+      final Multimap<ByteBuffer, LoadQueueItem> regionGroups) {
+    for (Map.Entry<ByteBuffer, Collection<LoadQueueItem>> e : regionGroups.asMap().entrySet()) {
+      Map<byte[], MutableInt> filesMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+      for (LoadQueueItem lqi : e.getValue()) {
+        MutableInt count = filesMap.computeIfAbsent(lqi.getFamily(), k -> new MutableInt());
+        count.increment();
+        if (count.intValue() > maxFilesPerRegionPerFamily) {
+          LOG.error("Trying to load more than " + maxFilesPerRegionPerFamily +
+            " hfiles to family " + Bytes.toStringBinary(lqi.getFamily()) +
+            " of region with start key " + Bytes.toStringBinary(e.getKey()));
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
+   * @param table the table to load into
+   * @param pool the ExecutorService
+   * @param queue the queue for LoadQueueItem
+   * @param startEndKeys start and end keys
+   * @return A map that groups LQI by likely bulk load region targets and Set of missing hfiles.
+   */
+  private Pair<Multimap<ByteBuffer, LoadQueueItem>, Set<String>> groupOrSplitPhase(
+      AsyncClusterConnection conn, TableName tableName, ExecutorService pool,
+      Deque<LoadQueueItem> queue, List<Pair<byte[], byte[]>> startEndKeys) throws IOException {
+    // <region start key, LQI> need synchronized only within this scope of this
+    // phase because of the puts that happen in futures.
+    Multimap<ByteBuffer, LoadQueueItem> rgs = HashMultimap.create();
+    final Multimap<ByteBuffer, LoadQueueItem> regionGroups = Multimaps.synchronizedMultimap(rgs);
+    Set<String> missingHFiles = new HashSet<>();
+    Pair<Multimap<ByteBuffer, LoadQueueItem>, Set<String>> pair =
+      new Pair<>(regionGroups, missingHFiles);
+
+    // drain LQIs and figure out bulk load groups
+    Set<Future<Pair<List<LoadQueueItem>, String>>> splittingFutures = new HashSet<>();
+    while (!queue.isEmpty()) {
+      final LoadQueueItem item = queue.remove();
+
+      final Callable<Pair<List<LoadQueueItem>, String>> call =
+        new Callable<Pair<List<LoadQueueItem>, String>>() {
+          @Override
+          public Pair<List<LoadQueueItem>, String> call() throws Exception {
+            Pair<List<LoadQueueItem>, String> splits =
+              groupOrSplit(conn, tableName, regionGroups, item, startEndKeys);
+            return splits;
+          }
+        };
+      splittingFutures.add(pool.submit(call));
+    }
+    // get all the results. All grouping and splitting must finish before
+    // we can attempt the atomic loads.
+    for (Future<Pair<List<LoadQueueItem>, String>> lqis : splittingFutures) {
+      try {
+        Pair<List<LoadQueueItem>, String> splits = lqis.get();
+        if (splits != null) {
+          if (splits.getFirst() != null) {
+            queue.addAll(splits.getFirst());
+          } else {
+            missingHFiles.add(splits.getSecond());
+          }
+        }
+      } catch (ExecutionException e1) {
+        Throwable t = e1.getCause();
+        if (t instanceof IOException) {
+          LOG.error("IOException during splitting", e1);
+          throw (IOException) t; // would have been thrown if not parallelized,
+        }
+        LOG.error("Unexpected execution exception during splitting", e1);
+        throw new IllegalStateException(t);
+      } catch (InterruptedException e1) {
+        LOG.error("Unexpected interrupted exception during splitting", e1);
+        throw (InterruptedIOException) new InterruptedIOException().initCause(e1);
+      }
+    }
+    return pair;
+  }
+
+  // unique file name for the table
+  private String getUniqueName() {
+    return UUID.randomUUID().toString().replaceAll("-", "");
+  }
+
+  private List<LoadQueueItem> splitStoreFile(LoadQueueItem item, TableDescriptor tableDesc,
+      byte[] startKey, byte[] splitKey) throws IOException {
+    Path hfilePath = item.getFilePath();
+    byte[] family = item.getFamily();
+    Path tmpDir = hfilePath.getParent();
+    if (!tmpDir.getName().equals(TMP_DIR)) {
+      tmpDir = new Path(tmpDir, TMP_DIR);
+    }
+
+    LOG.info("HFile at " + hfilePath + " no longer fits inside a single " + "region. Splitting...");
+
+    String uniqueName = getUniqueName();
+    ColumnFamilyDescriptor familyDesc = tableDesc.getColumnFamily(family);
+
+    Path botOut = new Path(tmpDir, uniqueName + ".bottom");
+    Path topOut = new Path(tmpDir, uniqueName + ".top");
+    splitStoreFile(getConf(), hfilePath, familyDesc, splitKey, botOut, topOut);
+
+    FileSystem fs = tmpDir.getFileSystem(getConf());
+    fs.setPermission(tmpDir, FsPermission.valueOf("-rwxrwxrwx"));
+    fs.setPermission(botOut, FsPermission.valueOf("-rwxrwxrwx"));
+    fs.setPermission(topOut, FsPermission.valueOf("-rwxrwxrwx"));
+
+    // Add these back at the *front* of the queue, so there's a lower
+    // chance that the region will just split again before we get there.
+    List<LoadQueueItem> lqis = new ArrayList<>(2);
+    lqis.add(new LoadQueueItem(family, botOut));
+    lqis.add(new LoadQueueItem(family, topOut));
+
+    // If the current item is already the result of previous splits,
+    // we don't need it anymore. Clean up to save space.
+    // It is not part of the original input files.
+    try {
+      if (tmpDir.getName().equals(TMP_DIR)) {
+        fs.delete(hfilePath, false);
+      }
+    } catch (IOException e) {
+      LOG.warn("Unable to delete temporary split file " + hfilePath);
+    }
+    LOG.info("Successfully split into new HFiles " + botOut + " and " + topOut);
+    return lqis;
+  }
+
+  /**
+   * Attempt to assign the given load queue item into its target region group. If the hfile boundary
+   * no longer fits into a region, physically splits the hfile such that the new bottom half will
+   * fit and returns the list of LQI's corresponding to the resultant hfiles.
+   * <p/>
+   * protected for testing
+   * @throws IOException if an IO failure is encountered
+   */
+  @VisibleForTesting
+  protected Pair<List<LoadQueueItem>, String> groupOrSplit(AsyncClusterConnection conn,
+      TableName tableName, Multimap<ByteBuffer, LoadQueueItem> regionGroups, LoadQueueItem item,
+      List<Pair<byte[], byte[]>> startEndKeys) throws IOException {
+    Path hfilePath = item.getFilePath();
+    Optional<byte[]> first, last;
+    try (HFile.Reader hfr = HFile.createReader(hfilePath.getFileSystem(getConf()), hfilePath,
+      CacheConfig.DISABLED, true, getConf())) {
+      hfr.loadFileInfo();
+      first = hfr.getFirstRowKey();
+      last = hfr.getLastRowKey();
+    } catch (FileNotFoundException fnfe) {
+      LOG.debug("encountered", fnfe);
+      return new Pair<>(null, hfilePath.getName());
+    }
+
+    LOG.info("Trying to load hfile=" + hfilePath + " first=" + first.map(Bytes::toStringBinary) +
+      " last=" + last.map(Bytes::toStringBinary));
+    if (!first.isPresent() || !last.isPresent()) {
+      assert !first.isPresent() && !last.isPresent();
+      // TODO what if this is due to a bad HFile?
+      LOG.info("hfile " + hfilePath + " has no entries, skipping");
+      return null;
+    }
+    if (Bytes.compareTo(first.get(), last.get()) > 0) {
+      throw new IllegalArgumentException("Invalid range: " + Bytes.toStringBinary(first.get()) +
+        " > " + Bytes.toStringBinary(last.get()));
+    }
+    int idx =
+      Collections.binarySearch(startEndKeys, Pair.newPair(first.get(), HConstants.EMPTY_END_ROW),
+        (p1, p2) -> Bytes.compareTo(p1.getFirst(), p2.getFirst()));
+    if (idx < 0) {
+      // not on boundary, returns -(insertion index). Calculate region it
+      // would be in.
+      idx = -(idx + 1) - 1;
+    }
+    int indexForCallable = idx;
+
+    /**
+     * we can consider there is a region hole in following conditions. 1) if idx < 0,then first
+     * region info is lost. 2) if the endkey of a region is not equal to the startkey of the next
+     * region. 3) if the endkey of the last region is not empty.
+     */
+    if (indexForCallable < 0) {
+      throw new IOException("The first region info for table " + tableName +
+        " can't be found in hbase:meta.Please use hbck tool to fix it first.");
+    } else if ((indexForCallable == startEndKeys.size() - 1) &&
+      !Bytes.equals(startEndKeys.get(indexForCallable).getSecond(), HConstants.EMPTY_BYTE_ARRAY)) {
+      throw new IOException("The last region info for table " + tableName +
+        " can't be found in hbase:meta.Please use hbck tool to fix it first.");
+    } else if (indexForCallable + 1 < startEndKeys.size() &&
+      !(Bytes.compareTo(startEndKeys.get(indexForCallable).getSecond(),
+        startEndKeys.get(indexForCallable + 1).getFirst()) == 0)) {
+      throw new IOException("The endkey of one region for table " + tableName +
+        " is not equal to the startkey of the next region in hbase:meta." +
+        "Please use hbck tool to fix it first.");
+    }
+
+    boolean lastKeyInRange = Bytes.compareTo(last.get(), startEndKeys.get(idx).getSecond()) < 0 ||
+      Bytes.equals(startEndKeys.get(idx).getSecond(), HConstants.EMPTY_BYTE_ARRAY);
+    if (!lastKeyInRange) {
+      Pair<byte[], byte[]> startEndKey = startEndKeys.get(indexForCallable);
+      List<LoadQueueItem> lqis =
+        splitStoreFile(item, FutureUtils.get(conn.getAdmin().getDescriptor(tableName)),
+          startEndKey.getFirst(), startEndKey.getSecond());
+      return new Pair<>(lqis, null);
+    }
+
+    // group regions.
+    regionGroups.put(ByteBuffer.wrap(startEndKeys.get(idx).getFirst()), item);
+    return null;
+  }
+
+  /**
+   * Split a storefile into a top and bottom half, maintaining the metadata, recreating bloom
+   * filters, etc.
+   */
+  @VisibleForTesting
+  static void splitStoreFile(Configuration conf, Path inFile, ColumnFamilyDescriptor familyDesc,
+      byte[] splitKey, Path bottomOut, Path topOut) throws IOException {
+    // Open reader with no block cache, and not in-memory
+    Reference topReference = Reference.createTopReference(splitKey);
+    Reference bottomReference = Reference.createBottomReference(splitKey);
+
+    copyHFileHalf(conf, inFile, topOut, topReference, familyDesc);
+    copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc);
+  }
+
+  /**
+   * Copy half of an HFile into a new HFile.
+   */
+  private static void copyHFileHalf(Configuration conf, Path inFile, Path outFile,
+      Reference reference, ColumnFamilyDescriptor familyDescriptor) throws IOException {
+    FileSystem fs = inFile.getFileSystem(conf);
+    CacheConfig cacheConf = CacheConfig.DISABLED;
+    HalfStoreFileReader halfReader = null;
+    StoreFileWriter halfWriter = null;
+    try {
+      halfReader = new HalfStoreFileReader(fs, inFile, cacheConf, reference, true,
+        new AtomicInteger(0), true, conf);
+      Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
+
+      int blocksize = familyDescriptor.getBlocksize();
+      Algorithm compression = familyDescriptor.getCompressionType();
+      BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
+      HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)
+        .withChecksumType(HStore.getChecksumType(conf))
+        .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf)).withBlockSize(blocksize)
+        .withDataBlockEncoding(familyDescriptor.getDataBlockEncoding()).withIncludesTags(true)
+        .build();
+      halfWriter = new StoreFileWriter.Builder(conf, cacheConf, fs).withFilePath(outFile)
+        .withBloomType(bloomFilterType).withFileContext(hFileContext).build();
+      HFileScanner scanner = halfReader.getScanner(false, false, false);
+      scanner.seekTo();
+      do {
+        halfWriter.append(scanner.getCell());
+      } while (scanner.next());
+
+      for (Map.Entry<byte[], byte[]> entry : fileInfo.entrySet()) {
+        if (shouldCopyHFileMetaKey(entry.getKey())) {
+          halfWriter.appendFileInfo(entry.getKey(), entry.getValue());
+        }
+      }
+    } finally {
+      if (halfReader != null) {
+        try {
+          halfReader.close(cacheConf.shouldEvictOnClose());
+        } catch (IOException e) {
+          LOG.warn("failed to close hfile reader for " + inFile, e);
+        }
+      }
+      if (halfWriter != null) {
+        halfWriter.close();
+      }
+    }
+  }
+
+  /**
+   * Infers region boundaries for a new table.
+   * <p/>
+   * Parameter: <br/>
+   * bdryMap is a map between keys to an integer belonging to {+1, -1}
+   * <ul>
+   * <li>If a key is a start key of a file, then it maps to +1</li>
+   * <li>If a key is an end key of a file, then it maps to -1</li>
+   * </ul>
+   * <p>
+   * Algo:<br/>
+   * <ol>
+   * <li>Poll on the keys in order:
+   * <ol type="a">
+   * <li>Keep adding the mapped values to these keys (runningSum)</li>
+   * <li>Each time runningSum reaches 0, add the start Key from when the runningSum had started to a
+   * boundary list.</li>
+   * </ol>
+   * </li>
+   * <li>Return the boundary list.</li>
+   * </ol>
+   */
+  public static byte[][] inferBoundaries(SortedMap<byte[], Integer> bdryMap) {
+    List<byte[]> keysArray = new ArrayList<>();
+    int runningValue = 0;
+    byte[] currStartKey = null;
+    boolean firstBoundary = true;
+
+    for (Map.Entry<byte[], Integer> item : bdryMap.entrySet()) {
+      if (runningValue == 0) {
+        currStartKey = item.getKey();
+      }
+      runningValue += item.getValue();
+      if (runningValue == 0) {
+        if (!firstBoundary) {
+          keysArray.add(currStartKey);
+        }
+        firstBoundary = false;
+      }
+    }
+
+    return keysArray.toArray(new byte[0][]);
+  }
+
+  /**
+   * If the table is created for the first time, then "completebulkload" reads the files twice. More
+   * modifications necessary if we want to avoid doing it.
+   */
+  private void createTable(TableName tableName, Path hfofDir, AsyncAdmin admin) throws IOException {
+    final FileSystem fs = hfofDir.getFileSystem(getConf());
+
+    // Add column families
+    // Build a set of keys
+    List<ColumnFamilyDescriptorBuilder> familyBuilders = new ArrayList<>();
+    SortedMap<byte[], Integer> map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    visitBulkHFiles(fs, hfofDir, new BulkHFileVisitor<ColumnFamilyDescriptorBuilder>() {
+      @Override
+      public ColumnFamilyDescriptorBuilder bulkFamily(byte[] familyName) {
+        ColumnFamilyDescriptorBuilder builder =
+          ColumnFamilyDescriptorBuilder.newBuilder(familyName);
+        familyBuilders.add(builder);
+        return builder;
+      }
+
+      @Override
+      public void bulkHFile(ColumnFamilyDescriptorBuilder builder, FileStatus hfileStatus)
+          throws IOException {
+        Path hfile = hfileStatus.getPath();
+        try (HFile.Reader reader =
+          HFile.createReader(fs, hfile, CacheConfig.DISABLED, true, getConf())) {
+          if (builder.getCompressionType() != reader.getFileContext().getCompression()) {
+            builder.setCompressionType(reader.getFileContext().getCompression());
+            LOG.info("Setting compression " + reader.getFileContext().getCompression().name() +
+              " for family " + builder.getNameAsString());
+          }
+          reader.loadFileInfo();
+          byte[] first = reader.getFirstRowKey().get();
+          byte[] last = reader.getLastRowKey().get();
+
+          LOG.info("Trying to figure out region boundaries hfile=" + hfile + " first=" +
+            Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));
+
+          // To eventually infer start key-end key boundaries
+          Integer value = map.containsKey(first) ? map.get(first) : 0;
+          map.put(first, value + 1);
+
+          value = map.containsKey(last) ? map.get(last) : 0;
+          map.put(last, value - 1);
+        }
+      }
+    }, true);
+
+    byte[][] keys = inferBoundaries(map);
+    TableDescriptorBuilder tdBuilder = TableDescriptorBuilder.newBuilder(tableName);
+    familyBuilders.stream().map(ColumnFamilyDescriptorBuilder::build)
+      .forEachOrdered(tdBuilder::setColumnFamily);
+    FutureUtils.get(admin.createTable(tdBuilder.build(), keys));
+
+    LOG.info("Table " + tableName + " is available!!");
+  }
+
+  private Map<LoadQueueItem, ByteBuffer> performBulkLoad(AsyncClusterConnection conn,
+      TableName tableName, Deque<LoadQueueItem> queue, ExecutorService pool, boolean copyFile)
+      throws IOException {
+    int count = 0;
+
+    fsDelegationToken.acquireDelegationToken(queue.peek().getFilePath().getFileSystem(getConf()));
+    bulkToken = FutureUtils.get(conn.prepareBulkLoad(tableName));
+    Pair<Multimap<ByteBuffer, LoadQueueItem>, Set<String>> pair = null;
+
+    Map<LoadQueueItem, ByteBuffer> item2RegionMap = new HashMap<>();
+    // Assumes that region splits can happen while this occurs.
+    while (!queue.isEmpty()) {
+      // need to reload split keys each iteration.
+      final List<Pair<byte[], byte[]>> startEndKeys =
+        FutureUtils.get(conn.getRegionLocator(tableName).getStartEndKeys());
+      if (count != 0) {
+        LOG.info("Split occurred while grouping HFiles, retry attempt " + +count + " with " +
+          queue.size() + " files remaining to group or split");
+      }
+
+      int maxRetries = getConf().getInt(HConstants.BULKLOAD_MAX_RETRIES_NUMBER, 10);
+      maxRetries = Math.max(maxRetries, startEndKeys.size() + 1);
+      if (maxRetries != 0 && count >= maxRetries) {
+        throw new IOException(
+          "Retry attempted " + count + " times without completing, bailing out");
+      }
+      count++;
+
+      // Using ByteBuffer for byte[] equality semantics
+      pair = groupOrSplitPhase(conn, tableName, pool, queue, startEndKeys);
+      Multimap<ByteBuffer, LoadQueueItem> regionGroups = pair.getFirst();
+
+      if (!checkHFilesCountPerRegionPerFamily(regionGroups)) {
+        // Error is logged inside checkHFilesCountPerRegionPerFamily.
+        throw new IOException("Trying to load more than " + maxFilesPerRegionPerFamily +
+          " hfiles to one family of one region");
+      }
+
+      bulkLoadPhase(conn, tableName, queue, regionGroups, copyFile, item2RegionMap);
+
+      // NOTE: The next iteration's split / group could happen in parallel to
+      // atomic bulkloads assuming that there are splits and no merges, and
+      // that we can atomically pull out the groups we want to retry.
+    }
+
+    if (!queue.isEmpty()) {
+      throw new RuntimeException(
+        "Bulk load aborted with some files not yet loaded." + "Please check log for more details.");
+    }
+    return item2RegionMap;
+  }
+
+  private void cleanup(AsyncClusterConnection conn, TableName tableName, Deque<LoadQueueItem> queue,
+      ExecutorService pool) throws IOException {
+    fsDelegationToken.releaseDelegationToken();
+    if (bulkToken != null) {
+      conn.cleanupBulkLoad(tableName, bulkToken);
+    }
+    if (pool != null) {
+      pool.shutdown();
+    }
+    if (!queue.isEmpty()) {
+      StringBuilder err = new StringBuilder();
+      err.append("-------------------------------------------------\n");
+      err.append("Bulk load aborted with some files not yet loaded:\n");
+      err.append("-------------------------------------------------\n");
+      for (LoadQueueItem q : queue) {
+        err.append("  ").append(q.getFilePath()).append('\n');
+      }
+      LOG.error(err.toString());
+    }
   }
 
-  private Map<BulkLoadHFiles.LoadQueueItem, ByteBuffer> convert(
-      Map<LoadIncrementalHFiles.LoadQueueItem, ByteBuffer> map) {
-    return map.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue()));
+  /**
+   * Perform a bulk load of the given directory into the given pre-existing table. This method is
+   * not threadsafe.
+   * @param map map of family to List of hfiles
+   * @param tableName table to load the hfiles
+   * @param silence true to ignore unmatched column families
+   * @param copyFile always copy hfiles if true
+   * @throws TableNotFoundException if table does not yet exist
+   */
+  private Map<LoadQueueItem, ByteBuffer> doBulkLoad(AsyncClusterConnection conn,
+      TableName tableName, Map<byte[], List<Path>> map, boolean silence, boolean copyFile)
+      throws TableNotFoundException, IOException {
+    if (!FutureUtils.get(conn.getAdmin().isTableAvailable(tableName))) {
+      throw new TableNotFoundException("Table " + tableName + " is not currently available.");
+    }
+    // LQI queue does not need to be threadsafe -- all operations on this queue
+    // happen in this thread
+    Deque<LoadQueueItem> queue = new ArrayDeque<>();
+    ExecutorService pool = null;
+    try {
+      prepareHFileQueue(conn, tableName, map, queue, silence);
+      if (queue.isEmpty()) {
+        LOG.warn("Bulk load operation did not get any files to load");
+        return Collections.emptyMap();
+      }
+      pool = createExecutorService();
+      return performBulkLoad(conn, tableName, queue, pool, copyFile);
+    } finally {
+      cleanup(conn, tableName, queue, pool);
+    }
+  }
+
+  /**
+   * Perform a bulk load of the given directory into the given pre-existing table. This method is
+   * not threadsafe.
+   * @param tableName table to load the hfiles
+   * @param hfofDir the directory that was provided as the output path of a job using
+   *          HFileOutputFormat
+   * @param silence true to ignore unmatched column families
+   * @param copyFile always copy hfiles if true
+   * @throws TableNotFoundException if table does not yet exist
+   */
+  private Map<LoadQueueItem, ByteBuffer> doBulkLoad(AsyncClusterConnection conn,
+      TableName tableName, Path hfofDir, boolean silence, boolean copyFile)
+      throws TableNotFoundException, IOException {
+    if (!FutureUtils.get(conn.getAdmin().isTableAvailable(tableName))) {
+      throw new TableNotFoundException("Table " + tableName + " is not currently available.");
+    }
+
+    /*
+     * Checking hfile format is a time-consuming operation, we should have an option to skip this
+     * step when bulkloading millions of HFiles. See HBASE-13985.
+     */
+    boolean validateHFile = getConf().getBoolean("hbase.loadincremental.validate.hfile", true);
+    if (!validateHFile) {
+      LOG.warn("You are skipping HFiles validation, it might cause some data loss if files " +
+        "are not correct. If you fail to read data from your table after using this " +
+        "option, consider removing the files and bulkload again without this option. " +
+        "See HBASE-13985");
+    }
+    // LQI queue does not need to be threadsafe -- all operations on this queue
+    // happen in this thread
+    Deque<LoadQueueItem> queue = new ArrayDeque<>();
+    ExecutorService pool = null;
+    try {
+      prepareHFileQueue(getConf(), conn, tableName, hfofDir, queue, validateHFile, silence);
+
+      if (queue.isEmpty()) {
+        LOG.warn(
+          "Bulk load operation did not find any files to load in " + "directory " + hfofDir != null
+            ? hfofDir.toUri().toString()
+            : "" + ".  Does it contain files in " +
+              "subdirectories that correspond to column family names?");
+        return Collections.emptyMap();
+      }
+      pool = createExecutorService();
+      return performBulkLoad(conn, tableName, queue, pool, copyFile);
+    } finally {
+      cleanup(conn, tableName, queue, pool);
+    }
   }
 
   @Override
-  public Map<BulkLoadHFiles.LoadQueueItem, ByteBuffer> bulkLoad(TableName tableName,
+  public Map<LoadQueueItem, ByteBuffer> bulkLoad(TableName tableName,
       Map<byte[], List<Path>> family2Files) throws TableNotFoundException, IOException {
-    return convert(run(family2Files, tableName));
+    try (AsyncClusterConnection conn = ClusterConnectionFactory
+      .createAsyncClusterConnection(getConf(), null, userProvider.getCurrent())) {
+      if (!FutureUtils.get(conn.getAdmin().tableExists(tableName))) {
+        String errorMsg = format("Table '%s' does not exist.", tableName);
+        LOG.error(errorMsg);
+        throw new TableNotFoundException(errorMsg);
+      }
+      return doBulkLoad(conn, tableName, family2Files, isSilence(), isAlwaysCopyFiles());
+    }
   }
 
   @Override
-  public Map<BulkLoadHFiles.LoadQueueItem, ByteBuffer> bulkLoad(TableName tableName, Path dir)
+  public Map<LoadQueueItem, ByteBuffer> bulkLoad(TableName tableName, Path dir)
       throws TableNotFoundException, IOException {
-    return convert(run(dir, tableName));
+    try (AsyncClusterConnection conn = ClusterConnectionFactory
+      .createAsyncClusterConnection(getConf(), null, userProvider.getCurrent())) {
+      AsyncAdmin admin = conn.getAdmin();
+      if (!FutureUtils.get(admin.tableExists(tableName))) {
+        if (isCreateTable()) {
+          createTable(tableName, dir, admin);
+        } else {
+          String errorMsg = format("Table '%s' does not exist.", tableName);
+          LOG.error(errorMsg);
+          throw new TableNotFoundException(errorMsg);
+        }
+      }
+      return doBulkLoad(conn, tableName, dir, isSilence(), isAlwaysCopyFiles());
+    }
+  }
+
+  public void setBulkToken(String bulkToken) {
+    this.bulkToken = bulkToken;
+  }
+
+  private void usage() {
+    System.err
+      .println("usage: " + NAME + " /path/to/hfileoutputformat-output tablename -loadTable" +
+        "\n -D" + CREATE_TABLE_CONF_KEY + "=no - can be used to avoid creation of table by " +
+        "this tool\n  Note: if you set this to 'no', then the target table must already exist " +
+        "in HBase\n -loadTable implies your baseDirectory to store file has a depth of 3 ,you" +
+        " must have an existing table\n-D" + IGNORE_UNMATCHED_CF_CONF_KEY + "=yes - can be used " +
+        "to ignore unmatched column families\n" + "\n");
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    if (args.length != 2 && args.length != 3) {
+      usage();
+      return -1;
+    }
+    Path dirPath = new Path(args[0]);
+    TableName tableName = TableName.valueOf(args[1]);
+
+    if (args.length == 2) {
+      return !bulkLoad(tableName, dirPath).isEmpty() ? 0 : -1;
+    } else {
+      Map<byte[], List<Path>> family2Files = Maps.newHashMap();
+      FileSystem fs = FileSystem.get(getConf());
+      for (FileStatus regionDir : fs.listStatus(dirPath)) {
+        FSVisitor.visitRegionStoreFiles(fs, regionDir.getPath(), (region, family, hfileName) -> {
+          Path path = new Path(regionDir.getPath(), new Path(family, hfileName));
+          byte[] familyName = Bytes.toBytes(family);
+          if (family2Files.containsKey(familyName)) {
+            family2Files.get(familyName).add(path);
+          } else {
+            family2Files.put(familyName, Lists.newArrayList(path));
+          }
+        });
+      }
+      return !bulkLoad(tableName, family2Files).isEmpty() ? 0 : -1;
+    }
   }
 
   public static void main(String[] args) throws Exception {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
deleted file mode 100644
index 82f5da4..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
+++ /dev/null
@@ -1,1283 +0,0 @@
-/**
- * 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.tool;
-
-import static java.lang.String.format;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Deque;
-import java.util.HashMap;
-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.SortedMap;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.Collectors;
-import org.apache.commons.lang3.mutable.MutableInt;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClientServiceCallable;
-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.RegionLocator;
-import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
-import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.io.HFileLink;
-import org.apache.hadoop.hbase.io.HalfStoreFileReader;
-import org.apache.hadoop.hbase.io.Reference;
-import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.io.hfile.HFileContext;
-import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
-import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
-import org.apache.hadoop.hbase.io.hfile.HFileScanner;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.regionserver.HStore;
-import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.security.token.FsDelegationToken;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSHDFSUtils;
-import org.apache.hadoop.hbase.util.FSVisitor;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-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.collect.HashMultimap;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
-import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
-import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
-import org.apache.hbase.thirdparty.com.google.common.collect.Multimaps;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * Tool to load the output of HFileOutputFormat into an existing table.
- * @deprecated since 2.2.0, will be removed in 3.0.0. Use {@link BulkLoadHFiles} instead. Please
- *             rewrite your code if you rely on methods other than the {@link #run(Map, TableName)}
- *             and {@link #run(String, TableName)}, as all the methods other than them will be
- *             removed with no replacement.
- */
-@Deprecated
-@InterfaceAudience.Public
-public class LoadIncrementalHFiles extends Configured implements Tool {
-
-  private static final Logger LOG = LoggerFactory.getLogger(LoadIncrementalHFiles.class);
-
-  /**
-   * @deprecated since 2.2.0, will be removed in 3.0.0, with no replacement. End user should not
-   *             depend on this value.
-   */
-  @Deprecated
-  public static final String NAME = BulkLoadHFilesTool.NAME;
-  static final String RETRY_ON_IO_EXCEPTION = BulkLoadHFiles.RETRY_ON_IO_EXCEPTION;
-  public static final String MAX_FILES_PER_REGION_PER_FAMILY =
-    BulkLoadHFiles.MAX_FILES_PER_REGION_PER_FAMILY;
-  private static final String ASSIGN_SEQ_IDS = BulkLoadHFiles.ASSIGN_SEQ_IDS;
-  public final static String CREATE_TABLE_CONF_KEY = BulkLoadHFiles.CREATE_TABLE_CONF_KEY;
-  public final static String IGNORE_UNMATCHED_CF_CONF_KEY =
-    BulkLoadHFiles.IGNORE_UNMATCHED_CF_CONF_KEY;
-  public final static String ALWAYS_COPY_FILES = BulkLoadHFiles.ALWAYS_COPY_FILES;
-
-  // We use a '.' prefix which is ignored when walking directory trees
-  // above. It is invalid family name.
-  static final String TMP_DIR = ".tmp";
-
-  private final int maxFilesPerRegionPerFamily;
-  private final boolean assignSeqIds;
-
-  // Source delegation token
-  private final FsDelegationToken fsDelegationToken;
-  private final UserProvider userProvider;
-  private final int nrThreads;
-  private AtomicInteger numRetries;
-  private final RpcControllerFactory rpcControllerFactory;
-
-  private String bulkToken;
-
-  /**
-   * Represents an HFile waiting to be loaded. An queue is used in this class in order to support
-   * the case where a region has split during the process of the load. When this happens, the HFile
-   * is split into two physical parts across the new region boundary, and each part is added back
-   * into the queue. The import process finishes when the queue is empty.
-   * @deprecated Use {@link BulkLoadHFiles} instead.
-   */
-  @InterfaceAudience.Public
-  @Deprecated
-  public static class LoadQueueItem extends BulkLoadHFiles.LoadQueueItem {
-
-    public LoadQueueItem(byte[] family, Path hfilePath) {
-      super(family, hfilePath);
-    }
-  }
-
-  public LoadIncrementalHFiles(Configuration conf) {
-    // make a copy, just to be sure we're not overriding someone else's config
-    super(HBaseConfiguration.create(conf));
-    conf = getConf();
-    // disable blockcache for tool invocation, see HBASE-10500
-    conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0);
-    userProvider = UserProvider.instantiate(conf);
-    fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
-    assignSeqIds = conf.getBoolean(ASSIGN_SEQ_IDS, true);
-    maxFilesPerRegionPerFamily = conf.getInt(MAX_FILES_PER_REGION_PER_FAMILY, 32);
-    nrThreads = conf.getInt("hbase.loadincremental.threads.max",
-      Runtime.getRuntime().availableProcessors());
-    numRetries = new AtomicInteger(0);
-    rpcControllerFactory = new RpcControllerFactory(conf);
-  }
-
-  private void usage() {
-    System.err.println("usage: " + NAME + " /path/to/hfileoutputformat-output tablename -loadTable"
-        + "\n -D" + CREATE_TABLE_CONF_KEY + "=no - can be used to avoid creation of table by "
-        + "this tool\n  Note: if you set this to 'no', then the target table must already exist "
-        + "in HBase\n -loadTable implies your baseDirectory to store file has a depth of 3 ,you"
-        + " must have an existing table\n-D" + IGNORE_UNMATCHED_CF_CONF_KEY + "=yes - can be used "
-        + "to ignore unmatched column families\n" +
-        "\n");
-  }
-
-  /**
-   * Prepare a collection of {@link LoadQueueItem} from list of source hfiles contained in the
-   * passed directory and validates whether the prepared queue has all the valid table column
-   * families in it.
-   * @param hfilesDir directory containing list of hfiles to be loaded into the table
-   * @param table table to which hfiles should be loaded
-   * @param queue queue which needs to be loaded into the table
-   * @param validateHFile if true hfiles will be validated for its format
-   * @throws IOException If any I/O or network error occurred
-   */
-  public void prepareHFileQueue(Path hfilesDir, Table table, Deque<LoadQueueItem> queue,
-      boolean validateHFile) throws IOException {
-    prepareHFileQueue(hfilesDir, table, queue, validateHFile, false);
-  }
-
-  /**
-   * Prepare a collection of {@link LoadQueueItem} from list of source hfiles contained in the
-   * passed directory and validates whether the prepared queue has all the valid table column
-   * families in it.
-   * @param hfilesDir directory containing list of hfiles to be loaded into the table
-   * @param table table to which hfiles should be loaded
-   * @param queue queue which needs to be loaded into the table
-   * @param validateHFile if true hfiles will be validated for its format
-   * @param silence true to ignore unmatched column families
-   * @throws IOException If any I/O or network error occurred
-   */
-  public void prepareHFileQueue(Path hfilesDir, Table table, Deque<LoadQueueItem> queue,
-      boolean validateHFile, boolean silence) throws IOException {
-    discoverLoadQueue(queue, hfilesDir, validateHFile);
-    validateFamiliesInHFiles(table, queue, silence);
-  }
-
-  /**
-   * Prepare a collection of {@link LoadQueueItem} from list of source hfiles contained in the
-   * passed directory and validates whether the prepared queue has all the valid table column
-   * families in it.
-   * @param map map of family to List of hfiles
-   * @param table table to which hfiles should be loaded
-   * @param queue queue which needs to be loaded into the table
-   * @param silence true to ignore unmatched column families
-   * @throws IOException If any I/O or network error occurred
-   */
-  public void prepareHFileQueue(Map<byte[], List<Path>> map, Table table,
-      Deque<LoadQueueItem> queue, boolean silence) throws IOException {
-    populateLoadQueue(queue, map);
-    validateFamiliesInHFiles(table, queue, silence);
-  }
-
-  /**
-   * Perform a bulk load of the given directory into the given pre-existing table. This method is
-   * not threadsafe.
-   * @param hfofDir the directory that was provided as the output path of a job using
-   *          HFileOutputFormat
-   * @param admin the Admin
-   * @param table the table to load into
-   * @param regionLocator region locator
-   * @throws TableNotFoundException if table does not yet exist
-   */
-  public Map<LoadQueueItem, ByteBuffer> doBulkLoad(Path hfofDir, final Admin admin, Table table,
-      RegionLocator regionLocator) throws TableNotFoundException, IOException {
-    return doBulkLoad(hfofDir, admin, table, regionLocator, false, false);
-  }
-
-  /**
-   * Perform a bulk load of the given directory into the given pre-existing table. This method is
-   * not threadsafe.
-   * @param map map of family to List of hfiles
-   * @param admin the Admin
-   * @param table the table to load into
-   * @param regionLocator region locator
-   * @param silence true to ignore unmatched column families
-   * @param copyFile always copy hfiles if true
-   * @throws TableNotFoundException if table does not yet exist
-   */
-  public Map<LoadQueueItem, ByteBuffer> doBulkLoad(Map<byte[], List<Path>> map, final Admin admin,
-      Table table, RegionLocator regionLocator, boolean silence, boolean copyFile)
-      throws TableNotFoundException, IOException {
-    if (!admin.isTableAvailable(regionLocator.getName())) {
-      throw new TableNotFoundException("Table " + table.getName() + " is not currently available.");
-    }
-    // LQI queue does not need to be threadsafe -- all operations on this queue
-    // happen in this thread
-    Deque<LoadQueueItem> queue = new ArrayDeque<>();
-    ExecutorService pool = null;
-    SecureBulkLoadClient secureClient = null;
-    try {
-      prepareHFileQueue(map, table, queue, silence);
-      if (queue.isEmpty()) {
-        LOG.warn("Bulk load operation did not get any files to load");
-        return Collections.emptyMap();
-      }
-      pool = createExecutorService();
-      secureClient = new SecureBulkLoadClient(table.getConfiguration(), table);
-      return performBulkLoad(admin, table, regionLocator, queue, pool, secureClient, copyFile);
-    } finally {
-      cleanup(admin, queue, pool, secureClient);
-    }
-  }
-
-  /**
-   * Perform a bulk load of the given directory into the given pre-existing table. This method is
-   * not threadsafe.
-   * @param hfofDir the directory that was provided as the output path of a job using
-   *          HFileOutputFormat
-   * @param admin the Admin
-   * @param table the table to load into
-   * @param regionLocator region locator
-   * @param silence true to ignore unmatched column families
-   * @param copyFile always copy hfiles if true
-   * @throws TableNotFoundException if table does not yet exist
-   */
-  public Map<LoadQueueItem, ByteBuffer> doBulkLoad(Path hfofDir, final Admin admin, Table table,
-      RegionLocator regionLocator, boolean silence, boolean copyFile)
-      throws TableNotFoundException, IOException {
-    if (!admin.isTableAvailable(regionLocator.getName())) {
-      throw new TableNotFoundException("Table " + table.getName() + " is not currently available.");
-    }
-
-    /*
-     * Checking hfile format is a time-consuming operation, we should have an option to skip this
-     * step when bulkloading millions of HFiles. See HBASE-13985.
-     */
-    boolean validateHFile = getConf().getBoolean("hbase.loadincremental.validate.hfile", true);
-    if (!validateHFile) {
-      LOG.warn("You are skipping HFiles validation, it might cause some data loss if files " +
-          "are not correct. If you fail to read data from your table after using this " +
-          "option, consider removing the files and bulkload again without this option. " +
-          "See HBASE-13985");
-    }
-    // LQI queue does not need to be threadsafe -- all operations on this queue
-    // happen in this thread
-    Deque<LoadQueueItem> queue = new ArrayDeque<>();
-    ExecutorService pool = null;
-    SecureBulkLoadClient secureClient = null;
-    try {
-      prepareHFileQueue(hfofDir, table, queue, validateHFile, silence);
-
-      if (queue.isEmpty()) {
-        LOG.warn(
-          "Bulk load operation did not find any files to load in " + "directory " + hfofDir != null
-              ? hfofDir.toUri().toString()
-              : "" + ".  Does it contain files in " +
-                  "subdirectories that correspond to column family names?");
-        return Collections.emptyMap();
-      }
-      pool = createExecutorService();
-      secureClient = new SecureBulkLoadClient(table.getConfiguration(), table);
-      return performBulkLoad(admin, table, regionLocator, queue, pool, secureClient, copyFile);
-    } finally {
-      cleanup(admin, queue, pool, secureClient);
-    }
-  }
-
-  /**
-   * Used by the replication sink to load the hfiles from the source cluster. It does the following,
-   * <ol>
-   * <li>LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)}</li>
-   * <li>LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)
-   * </li>
-   * </ol>
-   * @param table Table to which these hfiles should be loaded to
-   * @param conn Connection to use
-   * @param queue {@link LoadQueueItem} has hfiles yet to be loaded
-   * @param startEndKeys starting and ending row keys of the region
-   */
-  public void loadHFileQueue(Table table, Connection conn, Deque<LoadQueueItem> queue,
-      Pair<byte[][], byte[][]> startEndKeys) throws IOException {
-    loadHFileQueue(table, conn, queue, startEndKeys, false);
-  }
-
-  /**
-   * Used by the replication sink to load the hfiles from the source cluster. It does the following,
-   * <ol>
-   * <li>LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)}</li>
-   * <li>LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)
-   * </li>
-   * </ol>
-   * @param table Table to which these hfiles should be loaded to
-   * @param conn Connection to use
-   * @param queue {@link LoadQueueItem} has hfiles yet to be loaded
-   * @param startEndKeys starting and ending row keys of the region
-   */
-  public void loadHFileQueue(Table table, Connection conn, Deque<LoadQueueItem> queue,
-      Pair<byte[][], byte[][]> startEndKeys, boolean copyFile) throws IOException {
-    ExecutorService pool = null;
-    try {
-      pool = createExecutorService();
-      Multimap<ByteBuffer, LoadQueueItem> regionGroups =
-          groupOrSplitPhase(table, pool, queue, startEndKeys).getFirst();
-      bulkLoadPhase(table, conn, pool, queue, regionGroups, copyFile, null);
-    } finally {
-      if (pool != null) {
-        pool.shutdown();
-      }
-    }
-  }
-
-  private Map<LoadQueueItem, ByteBuffer> performBulkLoad(Admin admin, Table table,
-      RegionLocator regionLocator, Deque<LoadQueueItem> queue, ExecutorService pool,
-      SecureBulkLoadClient secureClient, boolean copyFile) throws IOException {
-    int count = 0;
-
-    fsDelegationToken.acquireDelegationToken(queue.peek().getFilePath().getFileSystem(getConf()));
-    bulkToken = secureClient.prepareBulkLoad(admin.getConnection());
-    Pair<Multimap<ByteBuffer, LoadQueueItem>, Set<String>> pair = null;
-
-    Map<LoadQueueItem, ByteBuffer> item2RegionMap = new HashMap<>();
-    // Assumes that region splits can happen while this occurs.
-    while (!queue.isEmpty()) {
-      // need to reload split keys each iteration.
-      final Pair<byte[][], byte[][]> startEndKeys = regionLocator.getStartEndKeys();
-      if (count != 0) {
-        LOG.info("Split occurred while grouping HFiles, retry attempt " + +count + " with " +
-            queue.size() + " files remaining to group or split");
-      }
-
-      int maxRetries = getConf().getInt(HConstants.BULKLOAD_MAX_RETRIES_NUMBER, 10);
-      maxRetries = Math.max(maxRetries, startEndKeys.getFirst().length + 1);
-      if (maxRetries != 0 && count >= maxRetries) {
-        throw new IOException(
-            "Retry attempted " + count + " times without completing, bailing out");
-      }
-      count++;
-
-      // Using ByteBuffer for byte[] equality semantics
-      pair = groupOrSplitPhase(table, pool, queue, startEndKeys);
-      Multimap<ByteBuffer, LoadQueueItem> regionGroups = pair.getFirst();
-
-      if (!checkHFilesCountPerRegionPerFamily(regionGroups)) {
-        // Error is logged inside checkHFilesCountPerRegionPerFamily.
-        throw new IOException("Trying to load more than " + maxFilesPerRegionPerFamily +
-            " hfiles to one family of one region");
-      }
-
-      bulkLoadPhase(table, admin.getConnection(), pool, queue, regionGroups, copyFile,
-        item2RegionMap);
-
-      // NOTE: The next iteration's split / group could happen in parallel to
-      // atomic bulkloads assuming that there are splits and no merges, and
-      // that we can atomically pull out the groups we want to retry.
-    }
-
-    if (!queue.isEmpty()) {
-      throw new RuntimeException("Bulk load aborted with some files not yet loaded." +
-          "Please check log for more details.");
-    }
-    return item2RegionMap;
-  }
-
-  /**
-   * This takes the LQI's grouped by likely regions and attempts to bulk load them. Any failures are
-   * re-queued for another pass with the groupOrSplitPhase.
-   * <p>
-   * protected for testing.
-   */
-  @VisibleForTesting
-  protected void bulkLoadPhase(Table table, Connection conn, ExecutorService pool,
-      Deque<LoadQueueItem> queue, Multimap<ByteBuffer, LoadQueueItem> regionGroups,
-      boolean copyFile, Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
-    // atomically bulk load the groups.
-    Set<Future<List<LoadQueueItem>>> loadingFutures = new HashSet<>();
-    for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> e : regionGroups.asMap()
-        .entrySet()) {
-      byte[] first = e.getKey().array();
-      Collection<LoadQueueItem> lqis = e.getValue();
-
-      ClientServiceCallable<byte[]> serviceCallable =
-          buildClientServiceCallable(conn, table.getName(), first, lqis, copyFile);
-
-      Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
-        @Override
-        public List<LoadQueueItem> call() throws Exception {
-          List<LoadQueueItem> toRetry =
-              tryAtomicRegionLoad(serviceCallable, table.getName(), first, lqis);
-          return toRetry;
-        }
-      };
-      if (item2RegionMap != null) {
-        for (LoadQueueItem lqi : lqis) {
-          item2RegionMap.put(lqi, e.getKey());
-        }
-      }
-      loadingFutures.add(pool.submit(call));
-    }
-
-    // get all the results.
-    for (Future<List<LoadQueueItem>> future : loadingFutures) {
-      try {
-        List<LoadQueueItem> toRetry = future.get();
-
-        if (item2RegionMap != null) {
-          for (LoadQueueItem lqi : toRetry) {
-            item2RegionMap.remove(lqi);
-          }
-        }
-        // LQIs that are requeued to be regrouped.
-        queue.addAll(toRetry);
-
-      } catch (ExecutionException e1) {
-        Throwable t = e1.getCause();
-        if (t instanceof IOException) {
-          // At this point something unrecoverable has happened.
-          // TODO Implement bulk load recovery
-          throw new IOException("BulkLoad encountered an unrecoverable problem", t);
-        }
-        LOG.error("Unexpected execution exception during bulk load", e1);
-        throw new IllegalStateException(t);
-      } catch (InterruptedException e1) {
-        LOG.error("Unexpected interrupted exception during bulk load", e1);
-        throw (InterruptedIOException) new InterruptedIOException().initCause(e1);
-      }
-    }
-  }
-
-  @VisibleForTesting
-  protected ClientServiceCallable<byte[]> buildClientServiceCallable(Connection conn,
-      TableName tableName, byte[] first, Collection<LoadQueueItem> lqis, boolean copyFile) {
-    List<Pair<byte[], String>> famPaths =
-        lqis.stream().map(lqi -> Pair.newPair(lqi.getFamily(), lqi.getFilePath().toString()))
-            .collect(Collectors.toList());
-    return new ClientServiceCallable<byte[]>(conn, tableName, first,
-        rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET) {
-      @Override
-      protected byte[] rpcCall() throws Exception {
-        SecureBulkLoadClient secureClient = null;
-        boolean success = false;
-        try {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Going to connect to server " + getLocation() + " for row " +
-                Bytes.toStringBinary(getRow()) + " with hfile group " +
-                LoadIncrementalHFiles.this.toString(famPaths));
-          }
-          byte[] regionName = getLocation().getRegionInfo().getRegionName();
-          try (Table table = conn.getTable(getTableName())) {
-            secureClient = new SecureBulkLoadClient(getConf(), table);
-            success = secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
-              assignSeqIds, fsDelegationToken.getUserToken(), bulkToken, copyFile);
-          }
-          return success ? regionName : null;
-        } finally {
-          // Best effort copying of files that might not have been imported
-          // from the staging directory back to original location
-          // in user directory
-          if (secureClient != null && !success) {
-            FileSystem targetFs = FileSystem.get(getConf());
-            FileSystem sourceFs = lqis.iterator().next().getFilePath().getFileSystem(getConf());
-            // Check to see if the source and target filesystems are the same
-            // If they are the same filesystem, we will try move the files back
-            // because previously we moved them to the staging directory.
-            if (FSHDFSUtils.isSameHdfs(getConf(), sourceFs, targetFs)) {
-              for (Pair<byte[], String> el : famPaths) {
-                Path hfileStagingPath = null;
-                Path hfileOrigPath = new Path(el.getSecond());
-                try {
-                  hfileStagingPath = new Path(new Path(bulkToken, Bytes.toString(el.getFirst())),
-                      hfileOrigPath.getName());
-                  if (targetFs.rename(hfileStagingPath, hfileOrigPath)) {
-                    LOG.debug("Moved back file " + hfileOrigPath + " from " + hfileStagingPath);
-                  } else if (targetFs.exists(hfileStagingPath)) {
-                    LOG.debug(
-                      "Unable to move back file " + hfileOrigPath + " from " + hfileStagingPath);
-                  }
-                } catch (Exception ex) {
-                  LOG.debug(
-                    "Unable to move back file " + hfileOrigPath + " from " + hfileStagingPath, ex);
-                }
-              }
-            }
-          }
-        }
-      }
-    };
-  }
-
-  private boolean checkHFilesCountPerRegionPerFamily(
-      final Multimap<ByteBuffer, LoadQueueItem> regionGroups) {
-    for (Map.Entry<ByteBuffer, Collection<LoadQueueItem>> e : regionGroups.asMap().entrySet()) {
-      Map<byte[], MutableInt> filesMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-      for (LoadQueueItem lqi : e.getValue()) {
-        MutableInt count = filesMap.computeIfAbsent(lqi.getFamily(), k -> new MutableInt());
-        count.increment();
-        if (count.intValue() > maxFilesPerRegionPerFamily) {
-          LOG.error("Trying to load more than " + maxFilesPerRegionPerFamily +
-              " hfiles to family " + Bytes.toStringBinary(lqi.getFamily()) +
-              " of region with start key " + Bytes.toStringBinary(e.getKey()));
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-
-  /**
-   * @param table the table to load into
-   * @param pool the ExecutorService
-   * @param queue the queue for LoadQueueItem
-   * @param startEndKeys start and end keys
-   * @return A map that groups LQI by likely bulk load region targets and Set of missing hfiles.
-   */
-  private Pair<Multimap<ByteBuffer, LoadQueueItem>, Set<String>> groupOrSplitPhase(
-      final Table table, ExecutorService pool, Deque<LoadQueueItem> queue,
-      final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
-    // <region start key, LQI> need synchronized only within this scope of this
-    // phase because of the puts that happen in futures.
-    Multimap<ByteBuffer, LoadQueueItem> rgs = HashMultimap.create();
-    final Multimap<ByteBuffer, LoadQueueItem> regionGroups = Multimaps.synchronizedMultimap(rgs);
-    Set<String> missingHFiles = new HashSet<>();
-    Pair<Multimap<ByteBuffer, LoadQueueItem>, Set<String>> pair =
-        new Pair<>(regionGroups, missingHFiles);
-
-    // drain LQIs and figure out bulk load groups
-    Set<Future<Pair<List<LoadQueueItem>, String>>> splittingFutures = new HashSet<>();
-    while (!queue.isEmpty()) {
-      final LoadQueueItem item = queue.remove();
-
-      final Callable<Pair<List<LoadQueueItem>, String>> call =
-          new Callable<Pair<List<LoadQueueItem>, String>>() {
-            @Override
-            public Pair<List<LoadQueueItem>, String> call() throws Exception {
-              Pair<List<LoadQueueItem>, String> splits =
-                  groupOrSplit(regionGroups, item, table, startEndKeys);
-              return splits;
-            }
-          };
-      splittingFutures.add(pool.submit(call));
-    }
-    // get all the results. All grouping and splitting must finish before
-    // we can attempt the atomic loads.
-    for (Future<Pair<List<LoadQueueItem>, String>> lqis : splittingFutures) {
-      try {
-        Pair<List<LoadQueueItem>, String> splits = lqis.get();
-        if (splits != null) {
-          if (splits.getFirst() != null) {
-            queue.addAll(splits.getFirst());
-          } else {
-            missingHFiles.add(splits.getSecond());
-          }
-        }
-      } catch (ExecutionException e1) {
-        Throwable t = e1.getCause();
-        if (t instanceof IOException) {
-          LOG.error("IOException during splitting", e1);
-          throw (IOException) t; // would have been thrown if not parallelized,
-        }
-        LOG.error("Unexpected execution exception during splitting", e1);
-        throw new IllegalStateException(t);
-      } catch (InterruptedException e1) {
-        LOG.error("Unexpected interrupted exception during splitting", e1);
-        throw (InterruptedIOException) new InterruptedIOException().initCause(e1);
-      }
-    }
-    return pair;
-  }
-
-  private List<LoadQueueItem> splitStoreFile(final LoadQueueItem item, final Table table,
-      byte[] startKey, byte[] splitKey) throws IOException {
-    Path hfilePath = item.getFilePath();
-    byte[] family = item.getFamily();
-    Path tmpDir = hfilePath.getParent();
-    if (!tmpDir.getName().equals(TMP_DIR)) {
-      tmpDir = new Path(tmpDir, TMP_DIR);
-    }
-
-    LOG.info("HFile at " + hfilePath + " no longer fits inside a single " + "region. Splitting...");
-
-    String uniqueName = getUniqueName();
-    ColumnFamilyDescriptor familyDesc = table.getDescriptor().getColumnFamily(family);
-
-    Path botOut = new Path(tmpDir, uniqueName + ".bottom");
-    Path topOut = new Path(tmpDir, uniqueName + ".top");
-    splitStoreFile(getConf(), hfilePath, familyDesc, splitKey, botOut, topOut);
-
-    FileSystem fs = tmpDir.getFileSystem(getConf());
-    fs.setPermission(tmpDir, FsPermission.valueOf("-rwxrwxrwx"));
-    fs.setPermission(botOut, FsPermission.valueOf("-rwxrwxrwx"));
-    fs.setPermission(topOut, FsPermission.valueOf("-rwxrwxrwx"));
-
-    // Add these back at the *front* of the queue, so there's a lower
-    // chance that the region will just split again before we get there.
-    List<LoadQueueItem> lqis = new ArrayList<>(2);
-    lqis.add(new LoadQueueItem(family, botOut));
-    lqis.add(new LoadQueueItem(family, topOut));
-
-    // If the current item is already the result of previous splits,
-    // we don't need it anymore. Clean up to save space.
-    // It is not part of the original input files.
-    try {
-      if (tmpDir.getName().equals(TMP_DIR)) {
-        fs.delete(hfilePath, false);
-      }
-    } catch (IOException e) {
-      LOG.warn("Unable to delete temporary split file " + hfilePath);
-    }
-    LOG.info("Successfully split into new HFiles " + botOut + " and " + topOut);
-    return lqis;
-  }
-
-  /**
-   * Attempt to assign the given load queue item into its target region group. If the hfile boundary
-   * no longer fits into a region, physically splits the hfile such that the new bottom half will
-   * fit and returns the list of LQI's corresponding to the resultant hfiles.
-   * <p>
-   * protected for testing
-   * @throws IOException if an IO failure is encountered
-   */
-  @VisibleForTesting
-  protected Pair<List<LoadQueueItem>, String> groupOrSplit(
-      Multimap<ByteBuffer, LoadQueueItem> regionGroups, final LoadQueueItem item, final Table table,
-      final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
-    Path hfilePath = item.getFilePath();
-    Optional<byte[]> first, last;
-    try (HFile.Reader hfr = HFile.createReader(hfilePath.getFileSystem(getConf()), hfilePath,
-      CacheConfig.DISABLED, true, getConf())) {
-      hfr.loadFileInfo();
-      first = hfr.getFirstRowKey();
-      last = hfr.getLastRowKey();
-    } catch (FileNotFoundException fnfe) {
-      LOG.debug("encountered", fnfe);
-      return new Pair<>(null, hfilePath.getName());
-    }
-
-    LOG.info("Trying to load hfile=" + hfilePath + " first=" + first.map(Bytes::toStringBinary) +
-        " last=" + last.map(Bytes::toStringBinary));
-    if (!first.isPresent() || !last.isPresent()) {
-      assert !first.isPresent() && !last.isPresent();
-      // TODO what if this is due to a bad HFile?
-      LOG.info("hfile " + hfilePath + " has no entries, skipping");
-      return null;
-    }
-    if (Bytes.compareTo(first.get(), last.get()) > 0) {
-      throw new IllegalArgumentException("Invalid range: " + Bytes.toStringBinary(first.get()) +
-          " > " + Bytes.toStringBinary(last.get()));
-    }
-    int idx = Arrays.binarySearch(startEndKeys.getFirst(), first.get(), Bytes.BYTES_COMPARATOR);
-    if (idx < 0) {
-      // not on boundary, returns -(insertion index). Calculate region it
-      // would be in.
-      idx = -(idx + 1) - 1;
-    }
-    int indexForCallable = idx;
-
-    /**
-     * we can consider there is a region hole in following conditions. 1) if idx < 0,then first
-     * region info is lost. 2) if the endkey of a region is not equal to the startkey of the next
-     * region. 3) if the endkey of the last region is not empty.
-     */
-    if (indexForCallable < 0) {
-      throw new IOException("The first region info for table " + table.getName() +
-          " can't be found in hbase:meta.Please use hbck tool to fix it first.");
-    } else if ((indexForCallable == startEndKeys.getFirst().length - 1) &&
-        !Bytes.equals(startEndKeys.getSecond()[indexForCallable], HConstants.EMPTY_BYTE_ARRAY)) {
-      throw new IOException("The last region info for table " + table.getName() +
-          " can't be found in hbase:meta.Please use hbck tool to fix it first.");
-    } else if (indexForCallable + 1 < startEndKeys.getFirst().length &&
-        !(Bytes.compareTo(startEndKeys.getSecond()[indexForCallable],
-          startEndKeys.getFirst()[indexForCallable + 1]) == 0)) {
-      throw new IOException("The endkey of one region for table " + table.getName() +
-          " is not equal to the startkey of the next region in hbase:meta." +
-          "Please use hbck tool to fix it first.");
-    }
-
-    boolean lastKeyInRange = Bytes.compareTo(last.get(), startEndKeys.getSecond()[idx]) < 0 ||
-        Bytes.equals(startEndKeys.getSecond()[idx], HConstants.EMPTY_BYTE_ARRAY);
-    if (!lastKeyInRange) {
-      List<LoadQueueItem> lqis = splitStoreFile(item, table,
-        startEndKeys.getFirst()[indexForCallable], startEndKeys.getSecond()[indexForCallable]);
-      return new Pair<>(lqis, null);
-    }
-
-    // group regions.
-    regionGroups.put(ByteBuffer.wrap(startEndKeys.getFirst()[idx]), item);
-    return null;
-  }
-
-  /**
-   * Attempts to do an atomic load of many hfiles into a region. If it fails, it returns a list of
-   * hfiles that need to be retried. If it is successful it will return an empty list.
-   * <p>
-   * NOTE: To maintain row atomicity guarantees, region server callable should succeed atomically
-   * and fails atomically.
-   * <p>
-   * Protected for testing.
-   * @return empty list if success, list of items to retry on recoverable failure
-   */
-  @VisibleForTesting
-  protected List<LoadQueueItem> tryAtomicRegionLoad(ClientServiceCallable<byte[]> serviceCallable,
-      final TableName tableName, final byte[] first, final Collection<LoadQueueItem> lqis)
-      throws IOException {
-    List<LoadQueueItem> toRetry = new ArrayList<>();
-    try {
-      Configuration conf = getConf();
-      byte[] region = RpcRetryingCallerFactory.instantiate(conf, null).<byte[]> newCaller()
-          .callWithRetries(serviceCallable, Integer.MAX_VALUE);
-      if (region == null) {
-        LOG.warn("Attempt to bulk load region containing " + Bytes.toStringBinary(first) +
-            " into table " + tableName + " with files " + lqis +
-            " failed.  This is recoverable and they will be retried.");
-        toRetry.addAll(lqis); // return lqi's to retry
-      }
-      // success
-      return toRetry;
-    } catch (IOException e) {
-      LOG.error("Encountered unrecoverable error from region server, additional details: " +
-                      serviceCallable.getExceptionMessageAdditionalDetail(),
-              e);
-      LOG.warn(
-              "Received a " + e.getClass().getSimpleName()
-                      + " from region server: "
-                      + serviceCallable.getExceptionMessageAdditionalDetail(), e);
-      if (getConf().getBoolean(RETRY_ON_IO_EXCEPTION, false)
-              && numRetries.get() < getConf().getInt(
-              HConstants.HBASE_CLIENT_RETRIES_NUMBER,
-              HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER)) {
-        LOG.warn("Will attempt to retry loading failed HFiles. Retry #"
-                + numRetries.incrementAndGet());
-        toRetry.addAll(lqis);
-        return toRetry;
-      }
-      LOG.error(RETRY_ON_IO_EXCEPTION + " is disabled. Unable to recover");
-      throw e;
-    }
-  }
-
-  /**
-   * If the table is created for the first time, then "completebulkload" reads the files twice. More
-   * modifications necessary if we want to avoid doing it.
-   */
-  private void createTable(TableName tableName, Path hfofDir, Admin admin) throws IOException {
-    final FileSystem fs = hfofDir.getFileSystem(getConf());
-
-    // Add column families
-    // Build a set of keys
-    List<ColumnFamilyDescriptorBuilder> familyBuilders = new ArrayList<>();
-    SortedMap<byte[], Integer> map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    visitBulkHFiles(fs, hfofDir, new BulkHFileVisitor<ColumnFamilyDescriptorBuilder>() {
-      @Override
-      public ColumnFamilyDescriptorBuilder bulkFamily(byte[] familyName) {
-        ColumnFamilyDescriptorBuilder builder =
-            ColumnFamilyDescriptorBuilder.newBuilder(familyName);
-        familyBuilders.add(builder);
-        return builder;
-      }
-
-      @Override
-      public void bulkHFile(ColumnFamilyDescriptorBuilder builder, FileStatus hfileStatus)
-          throws IOException {
-        Path hfile = hfileStatus.getPath();
-        try (HFile.Reader reader =
-            HFile.createReader(fs, hfile, CacheConfig.DISABLED, true, getConf())) {
-          if (builder.getCompressionType() != reader.getFileContext().getCompression()) {
-            builder.setCompressionType(reader.getFileContext().getCompression());
-            LOG.info("Setting compression " + reader.getFileContext().getCompression().name() +
-                " for family " + builder.getNameAsString());
-          }
-          reader.loadFileInfo();
-          byte[] first = reader.getFirstRowKey().get();
-          byte[] last = reader.getLastRowKey().get();
-
-          LOG.info("Trying to figure out region boundaries hfile=" + hfile + " first=" +
-              Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));
-
-          // To eventually infer start key-end key boundaries
-          Integer value = map.containsKey(first) ? map.get(first) : 0;
-          map.put(first, value + 1);
-
-          value = map.containsKey(last) ? map.get(last) : 0;
-          map.put(last, value - 1);
-        }
-      }
-    });
-
-    byte[][] keys = inferBoundaries(map);
-    TableDescriptorBuilder tdBuilder = TableDescriptorBuilder.newBuilder(tableName);
-    familyBuilders.stream().map(ColumnFamilyDescriptorBuilder::build)
-        .forEachOrdered(tdBuilder::setColumnFamily);
-    admin.createTable(tdBuilder.build(), keys);
-
-    LOG.info("Table " + tableName + " is available!!");
-  }
-
-  private void cleanup(Admin admin, Deque<LoadQueueItem> queue, ExecutorService pool,
-      SecureBulkLoadClient secureClient) throws IOException {
-    fsDelegationToken.releaseDelegationToken();
-    if (bulkToken != null && secureClient != null) {
-      secureClient.cleanupBulkLoad(admin.getConnection(), bulkToken);
-    }
-    if (pool != null) {
-      pool.shutdown();
-    }
-    if (!queue.isEmpty()) {
-      StringBuilder err = new StringBuilder();
-      err.append("-------------------------------------------------\n");
-      err.append("Bulk load aborted with some files not yet loaded:\n");
-      err.append("-------------------------------------------------\n");
-      for (LoadQueueItem q : queue) {
-        err.append("  ").append(q.getFilePath()).append('\n');
-      }
-      LOG.error(err.toString());
-    }
-  }
-
-  // unique file name for the table
-  private String getUniqueName() {
-    return UUID.randomUUID().toString().replaceAll("-", "");
-  }
-
-  /**
-   * Checks whether there is any invalid family name in HFiles to be bulk loaded.
-   */
-  private void validateFamiliesInHFiles(Table table, Deque<LoadQueueItem> queue, boolean silence)
-      throws IOException {
-    Set<String> familyNames = Arrays.asList(table.getDescriptor().getColumnFamilies()).stream()
-        .map(f -> f.getNameAsString()).collect(Collectors.toSet());
-    List<String> unmatchedFamilies = queue.stream().map(item -> Bytes.toString(item.getFamily()))
-        .filter(fn -> !familyNames.contains(fn)).distinct().collect(Collectors.toList());
-    if (unmatchedFamilies.size() > 0) {
-      String msg =
-          "Unmatched family names found: unmatched family names in HFiles to be bulkloaded: " +
-              unmatchedFamilies + "; valid family names of table " + table.getName() + " are: " +
-              familyNames;
-      LOG.error(msg);
-      if (!silence) {
-        throw new IOException(msg);
-      }
-    }
-  }
-
-  /**
-   * Populate the Queue with given HFiles
-   */
-  private void populateLoadQueue(Deque<LoadQueueItem> ret, Map<byte[], List<Path>> map) {
-    map.forEach((k, v) -> v.stream().map(p -> new LoadQueueItem(k, p)).forEachOrdered(ret::add));
-  }
-
-  /**
-   * Walk the given directory for all HFiles, and return a Queue containing all such files.
-   */
-  private void discoverLoadQueue(final Deque<LoadQueueItem> ret, final Path hfofDir,
-      final boolean validateHFile) throws IOException {
-    visitBulkHFiles(hfofDir.getFileSystem(getConf()), hfofDir, new BulkHFileVisitor<byte[]>() {
-      @Override
-      public byte[] bulkFamily(final byte[] familyName) {
-        return familyName;
-      }
-
-      @Override
-      public void bulkHFile(final byte[] family, final FileStatus hfile) throws IOException {
-        long length = hfile.getLen();
-        if (length > getConf().getLong(HConstants.HREGION_MAX_FILESIZE,
-          HConstants.DEFAULT_MAX_FILE_SIZE)) {
-          LOG.warn("Trying to bulk load hfile " + hfile.getPath() + " with size: " + length +
-              " bytes can be problematic as it may lead to oversplitting.");
-        }
-        ret.add(new LoadQueueItem(family, hfile.getPath()));
-      }
-    }, validateHFile);
-  }
-
-  private interface BulkHFileVisitor<TFamily> {
-
-    TFamily bulkFamily(byte[] familyName) throws IOException;
-
-    void bulkHFile(TFamily family, FileStatus hfileStatus) throws IOException;
-  }
-
-  /**
-   * Iterate over the bulkDir hfiles. Skip reference, HFileLink, files starting with "_" and
-   * non-valid hfiles.
-   */
-  private static <TFamily> void visitBulkHFiles(final FileSystem fs, final Path bulkDir,
-      final BulkHFileVisitor<TFamily> visitor) throws IOException {
-    visitBulkHFiles(fs, bulkDir, visitor, true);
-  }
-
-  /**
-   * Iterate over the bulkDir hfiles. Skip reference, HFileLink, files starting with "_". Check and
-   * skip non-valid hfiles by default, or skip this validation by setting
-   * 'hbase.loadincremental.validate.hfile' to false.
-   */
-  private static <TFamily> void visitBulkHFiles(FileSystem fs, Path bulkDir,
-      BulkHFileVisitor<TFamily> visitor, boolean validateHFile) throws IOException {
-    FileStatus[] familyDirStatuses = fs.listStatus(bulkDir);
-    for (FileStatus familyStat : familyDirStatuses) {
-      if (!familyStat.isDirectory()) {
-        LOG.warn("Skipping non-directory " + familyStat.getPath());
-        continue;
-      }
-      Path familyDir = familyStat.getPath();
-      byte[] familyName = Bytes.toBytes(familyDir.getName());
-      // Skip invalid family
-      try {
-        ColumnFamilyDescriptorBuilder.isLegalColumnFamilyName(familyName);
-      } catch (IllegalArgumentException e) {
-        LOG.warn("Skipping invalid " + familyStat.getPath());
-        continue;
-      }
-      TFamily family = visitor.bulkFamily(familyName);
-
-      FileStatus[] hfileStatuses = fs.listStatus(familyDir);
-      for (FileStatus hfileStatus : hfileStatuses) {
-        if (!fs.isFile(hfileStatus.getPath())) {
-          LOG.warn("Skipping non-file " + hfileStatus);
-          continue;
-        }
-
-        Path hfile = hfileStatus.getPath();
-        // Skip "_", reference, HFileLink
-        String fileName = hfile.getName();
-        if (fileName.startsWith("_")) {
-          continue;
-        }
-        if (StoreFileInfo.isReference(fileName)) {
-          LOG.warn("Skipping reference " + fileName);
-          continue;
-        }
-        if (HFileLink.isHFileLink(fileName)) {
-          LOG.warn("Skipping HFileLink " + fileName);
-          continue;
-        }
-
-        // Validate HFile Format if needed
-        if (validateHFile) {
-          try {
-            if (!HFile.isHFileFormat(fs, hfile)) {
-              LOG.warn("the file " + hfile + " doesn't seems to be an hfile. skipping");
-              continue;
-            }
-          } catch (FileNotFoundException e) {
-            LOG.warn("the file " + hfile + " was removed");
-            continue;
-          }
-        }
-
-        visitor.bulkHFile(family, hfileStatus);
-      }
-    }
-  }
-
-  // Initialize a thread pool
-  private ExecutorService createExecutorService() {
-    ThreadPoolExecutor pool = new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS,
-        new LinkedBlockingQueue<>(),
-        new ThreadFactoryBuilder().setNameFormat("LoadIncrementalHFiles-%1$d").build());
-    pool.allowCoreThreadTimeOut(true);
-    return pool;
-  }
-
-  private final String toString(List<Pair<byte[], String>> list) {
-    StringBuilder sb = new StringBuilder();
-    sb.append('[');
-    list.forEach(p -> {
-      sb.append('{').append(Bytes.toStringBinary(p.getFirst())).append(',').append(p.getSecond())
-          .append('}');
-    });
-    sb.append(']');
-    return sb.toString();
-  }
-
-  /**
-   * Split a storefile into a top and bottom half, maintaining the metadata, recreating bloom
-   * filters, etc.
-   */
-  @VisibleForTesting
-  static void splitStoreFile(Configuration conf, Path inFile, ColumnFamilyDescriptor familyDesc,
-      byte[] splitKey, Path bottomOut, Path topOut) throws IOException {
-    // Open reader with no block cache, and not in-memory
-    Reference topReference = Reference.createTopReference(splitKey);
-    Reference bottomReference = Reference.createBottomReference(splitKey);
-
-    copyHFileHalf(conf, inFile, topOut, topReference, familyDesc);
-    copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc);
-  }
-
-  /**
-   * Copy half of an HFile into a new HFile.
-   */
-  private static void copyHFileHalf(Configuration conf, Path inFile, Path outFile,
-      Reference reference, ColumnFamilyDescriptor familyDescriptor) throws IOException {
-    FileSystem fs = inFile.getFileSystem(conf);
-    CacheConfig cacheConf = CacheConfig.DISABLED;
-    HalfStoreFileReader halfReader = null;
-    StoreFileWriter halfWriter = null;
-    try {
-      halfReader = new HalfStoreFileReader(fs, inFile, cacheConf, reference, true,
-          new AtomicInteger(0), true, conf);
-      Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
-
-      int blocksize = familyDescriptor.getBlocksize();
-      Algorithm compression = familyDescriptor.getCompressionType();
-      BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
-      HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)
-          .withChecksumType(HStore.getChecksumType(conf))
-          .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf)).withBlockSize(blocksize)
-          .withDataBlockEncoding(familyDescriptor.getDataBlockEncoding()).withIncludesTags(true)
-          .build();
-      halfWriter = new StoreFileWriter.Builder(conf, cacheConf, fs).withFilePath(outFile)
-          .withBloomType(bloomFilterType).withFileContext(hFileContext).build();
-      HFileScanner scanner = halfReader.getScanner(false, false, false);
-      scanner.seekTo();
-      do {
-        halfWriter.append(scanner.getCell());
-      } while (scanner.next());
-
-      for (Map.Entry<byte[], byte[]> entry : fileInfo.entrySet()) {
-        if (shouldCopyHFileMetaKey(entry.getKey())) {
-          halfWriter.appendFileInfo(entry.getKey(), entry.getValue());
-        }
-      }
-    } finally {
-      if (halfReader != null) {
-        try {
-          halfReader.close(cacheConf.shouldEvictOnClose());
-        } catch (IOException e) {
-          LOG.warn("failed to close hfile reader for " + inFile, e);
-        }
-      }
-      if (halfWriter != null) {
-        halfWriter.close();
-      }
-
-    }
-  }
-
-  private static boolean shouldCopyHFileMetaKey(byte[] key) {
-    // skip encoding to keep hfile meta consistent with data block info, see HBASE-15085
-    if (Bytes.equals(key, HFileDataBlockEncoder.DATA_BLOCK_ENCODING)) {
-      return false;
-    }
-
-    return !HFile.isReservedFileInfoKey(key);
-  }
-
-  private boolean isCreateTable() {
-    return "yes".equalsIgnoreCase(getConf().get(CREATE_TABLE_CONF_KEY, "yes"));
-  }
-
-  private boolean isSilence() {
-    return "yes".equalsIgnoreCase(getConf().get(IGNORE_UNMATCHED_CF_CONF_KEY, ""));
-  }
-
-  private boolean isAlwaysCopyFiles() {
-    return getConf().getBoolean(ALWAYS_COPY_FILES, false);
-  }
-
-  protected final Map<LoadQueueItem, ByteBuffer> run(Path hfofDir, TableName tableName)
-      throws IOException {
-    try (Connection connection = ConnectionFactory.createConnection(getConf());
-        Admin admin = connection.getAdmin()) {
-      if (!admin.tableExists(tableName)) {
-        if (isCreateTable()) {
-          createTable(tableName, hfofDir, admin);
-        } else {
-          String errorMsg = format("Table '%s' does not exist.", tableName);
-          LOG.error(errorMsg);
-          throw new TableNotFoundException(errorMsg);
-        }
-      }
-      try (Table table = connection.getTable(tableName);
-          RegionLocator locator = connection.getRegionLocator(tableName)) {
-        return doBulkLoad(hfofDir, admin, table, locator, isSilence(),
-            isAlwaysCopyFiles());
-      }
-    }
-  }
-  /**
-   * Perform bulk load on the given table.
-   * @param hfofDir the directory that was provided as the output path of a job using
-   *          HFileOutputFormat
-   * @param tableName the table to load into
-   */
-  public Map<LoadQueueItem, ByteBuffer> run(String hfofDir, TableName tableName)
-      throws IOException {
-    return run(new Path(hfofDir), tableName);
-  }
-
-  /**
-   * Perform bulk load on the given table.
-   * @param family2Files map of family to List of hfiles
-   * @param tableName the table to load into
-   */
-  public Map<LoadQueueItem, ByteBuffer> run(Map<byte[], List<Path>> family2Files,
-      TableName tableName) throws IOException {
-    try (Connection connection = ConnectionFactory.createConnection(getConf());
-        Admin admin = connection.getAdmin()) {
-      if (!admin.tableExists(tableName)) {
-        String errorMsg = format("Table '%s' does not exist.", tableName);
-        LOG.error(errorMsg);
-        throw new TableNotFoundException(errorMsg);
-      }
-      try (Table table = connection.getTable(tableName);
-          RegionLocator locator = connection.getRegionLocator(tableName)) {
-        return doBulkLoad(family2Files, admin, table, locator, isSilence(), isAlwaysCopyFiles());
-      }
-    }
-  }
-
-  @Override
-  public int run(String[] args) throws Exception {
-    if (args.length != 2 && args.length != 3) {
-      usage();
-      return -1;
-    }
-    String dirPath = args[0];
-    TableName tableName = TableName.valueOf(args[1]);
-
-
-    if (args.length == 2) {
-      return !run(dirPath, tableName).isEmpty() ? 0 : -1;
-    } else {
-      Map<byte[], List<Path>> family2Files = Maps.newHashMap();
-      FileSystem fs = FileSystem.get(getConf());
-      for (FileStatus regionDir : fs.listStatus(new Path(dirPath))) {
-        FSVisitor.visitRegionStoreFiles(fs, regionDir.getPath(), (region, family, hfileName) -> {
-          Path path = new Path(regionDir.getPath(), new Path(family, hfileName));
-          byte[] familyName = Bytes.toBytes(family);
-          if (family2Files.containsKey(familyName)) {
-            family2Files.get(familyName).add(path);
-          } else {
-            family2Files.put(familyName, Lists.newArrayList(path));
-          }
-        });
-      }
-      return !run(family2Files, tableName).isEmpty() ? 0 : -1;
-    }
-
-  }
-
-  public static void main(String[] args) throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    int ret = ToolRunner.run(conf, new LoadIncrementalHFiles(conf), args);
-    System.exit(ret);
-  }
-
-  /**
-   * Called from replication sink, where it manages bulkToken(staging directory) by itself. This is
-   * used only when SecureBulkLoadEndpoint is configured in hbase.coprocessor.region.classes
-   * property. This directory is used as a temporary directory where all files are initially
-   * copied/moved from user given directory, set all the required file permissions and then from
-   * their it is finally loaded into a table. This should be set only when, one would like to manage
-   * the staging directory by itself. Otherwise this tool will handle this by itself.
-   * @param stagingDir staging directory path
-   */
-  public void setBulkToken(String stagingDir) {
-    this.bulkToken = stagingDir;
-  }
-
-  /**
-   * Infers region boundaries for a new table.
-   * <p>
-   * Parameter: <br>
-   * bdryMap is a map between keys to an integer belonging to {+1, -1}
-   * <ul>
-   * <li>If a key is a start key of a file, then it maps to +1</li>
-   * <li>If a key is an end key of a file, then it maps to -1</li>
-   * </ul>
-   * <p>
-   * Algo:<br>
-   * <ol>
-   * <li>Poll on the keys in order:
-   * <ol type="a">
-   * <li>Keep adding the mapped values to these keys (runningSum)</li>
-   * <li>Each time runningSum reaches 0, add the start Key from when the runningSum had started to a
-   * boundary list.</li>
-   * </ol>
-   * </li>
-   * <li>Return the boundary list.</li>
-   * </ol>
-   */
-  public static byte[][] inferBoundaries(SortedMap<byte[], Integer> bdryMap) {
-    List<byte[]> keysArray = new ArrayList<>();
-    int runningValue = 0;
-    byte[] currStartKey = null;
-    boolean firstBoundary = true;
-
-    for (Map.Entry<byte[], Integer> item : bdryMap.entrySet()) {
-      if (runningValue == 0) {
-        currStartKey = item.getKey();
-      }
-      runningValue += item.getValue();
-      if (runningValue == 0) {
-        if (!firstBoundary) {
-          keysArray.add(currStartKey);
-        }
-        firstBoundary = false;
-      }
-    }
-
-    return keysArray.toArray(new byte[0][]);
-  }
-}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 34c846d..dfaa16d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -126,6 +126,7 @@ import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
 import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
@@ -3562,14 +3563,12 @@ public class HBaseFsck extends Configured implements Closeable {
   }
 
   public void dumpSidelinedRegions(Map<Path, HbckInfo> regions) {
-    for (Map.Entry<Path, HbckInfo> entry: regions.entrySet()) {
+    for (Map.Entry<Path, HbckInfo> entry : regions.entrySet()) {
       TableName tableName = entry.getValue().getTableName();
       Path path = entry.getKey();
-      errors.print("This sidelined region dir should be bulk loaded: "
-        + path.toString());
-      errors.print("Bulk load command looks like: "
-        + "hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles "
-        + path.toUri().getPath() + " "+ tableName);
+      errors.print("This sidelined region dir should be bulk loaded: " + path.toString());
+      errors.print("Bulk load command looks like: " + BulkLoadHFilesTool.NAME + " " +
+        path.toUri().getPath() + " " + tableName);
     }
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncClusterConnection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncClusterConnection.java
new file mode 100644
index 0000000..d5fc58e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncClusterConnection.java
@@ -0,0 +1,155 @@
+/**
+ * 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 java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.security.token.Token;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
+
+/**
+ * Can be overridden in UT if you only want to implement part of the methods in
+ * {@link AsyncClusterConnection}.
+ */
+public class DummyAsyncClusterConnection implements AsyncClusterConnection {
+
+  @Override
+  public Configuration getConfiguration() {
+    return null;
+  }
+
+  @Override
+  public AsyncTableRegionLocator getRegionLocator(TableName tableName) {
+    return null;
+  }
+
+  @Override
+  public void clearRegionLocationCache() {
+  }
+
+  @Override
+  public AsyncTableBuilder<AdvancedScanResultConsumer> getTableBuilder(TableName tableName) {
+    return null;
+  }
+
+  @Override
+  public AsyncTableBuilder<ScanResultConsumer> getTableBuilder(TableName tableName,
+      ExecutorService pool) {
+    return null;
+  }
+
+  @Override
+  public AsyncAdminBuilder getAdminBuilder() {
+    return null;
+  }
+
+  @Override
+  public AsyncAdminBuilder getAdminBuilder(ExecutorService pool) {
+    return null;
+  }
+
+  @Override
+  public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName) {
+    return null;
+  }
+
+  @Override
+  public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName,
+      ExecutorService pool) {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<Hbck> getHbck() {
+    return null;
+  }
+
+  @Override
+  public Hbck getHbck(ServerName masterServer) throws IOException {
+    return null;
+  }
+
+  @Override
+  public boolean isClosed() {
+    return false;
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName) {
+    return null;
+  }
+
+  @Override
+  public NonceGenerator getNonceGenerator() {
+    return null;
+  }
+
+  @Override
+  public RpcClient getRpcClient() {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<FlushRegionResponse> flush(byte[] regionName,
+      boolean writeFlushWALMarker) {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<Long> replay(TableName tableName, byte[] encodedRegionName, byte[] row,
+      List<Entry> entries, int replicaId, int numRetries, long operationTimeoutNs) {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
+      boolean reload) {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<String> prepareBulkLoad(TableName tableName) {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<Boolean> bulkLoad(TableName tableName,
+      List<Pair<byte[], String>> familyPaths, byte[] row, boolean assignSeqNum, Token<?> userToken,
+      String bulkToken, boolean copyFiles) {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<Void> cleanupBulkLoad(TableName tableName, String bulkToken) {
+    return null;
+  }
+}
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/DummyAsyncRegistry.java
new file mode 100644
index 0000000..e9ae25d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncRegistry.java
@@ -0,0 +1,60 @@
+/**
+ * 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 java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+
+/**
+ * Can be overridden in UT if you only want to implement part of the methods in
+ * {@link AsyncRegistry}.
+ */
+public class DummyAsyncRegistry implements AsyncRegistry {
+
+  public static final String REGISTRY_IMPL_CONF_KEY = AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY;
+
+  @Override
+  public CompletableFuture<RegionLocations> getMetaRegionLocation() {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<String> getClusterId() {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<Integer> getCurrentNrHRS() {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<ServerName> getMasterAddress() {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<Integer> getMasterInfoPort() {
+    return null;
+  }
+
+  @Override
+  public void close() {
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncTable.java
new file mode 100644
index 0000000..d9d6915
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncTable.java
@@ -0,0 +1,150 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import com.google.protobuf.RpcChannel;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+
+/**
+ * Can be overridden in UT if you only want to implement part of the methods in {@link AsyncTable}.
+ */
+public class DummyAsyncTable<C extends ScanResultConsumerBase> implements AsyncTable<C> {
+
+  @Override
+  public TableName getName() {
+    return null;
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return null;
+  }
+
+  @Override
+  public long getRpcTimeout(TimeUnit unit) {
+    return 0;
+  }
+
+  @Override
+  public long getReadRpcTimeout(TimeUnit unit) {
+    return 0;
+  }
+
+  @Override
+  public long getWriteRpcTimeout(TimeUnit unit) {
+    return 0;
+  }
+
+  @Override
+  public long getOperationTimeout(TimeUnit unit) {
+    return 0;
+  }
+
+  @Override
+  public long getScanTimeout(TimeUnit unit) {
+    return 0;
+  }
+
+  @Override
+  public CompletableFuture<Result> get(Get get) {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<Void> put(Put put) {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<Void> delete(Delete delete) {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<Result> append(Append append) {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<Result> increment(Increment increment) {
+    return null;
+  }
+
+  @Override
+  public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<Void> mutateRow(RowMutations mutation) {
+    return null;
+  }
+
+  @Override
+  public void scan(Scan scan, C consumer) {
+  }
+
+  @Override
+  public ResultScanner getScanner(Scan scan) {
+    return null;
+  }
+
+  @Override
+  public CompletableFuture<List<Result>> scanAll(Scan scan) {
+    return null;
+  }
+
+  @Override
+  public List<CompletableFuture<Result>> get(List<Get> gets) {
+    return null;
+  }
+
+  @Override
+  public List<CompletableFuture<Void>> put(List<Put> puts) {
+    return null;
+  }
+
+  @Override
+  public List<CompletableFuture<Void>> delete(List<Delete> deletes) {
+    return null;
+  }
+
+  @Override
+  public <T> List<CompletableFuture<T>> batch(List<? extends Row> actions) {
+    return null;
+  }
+
+  @Override
+  public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
+      ServiceCaller<S, R> callable, byte[] row) {
+    return null;
+  }
+
+  @Override
+  public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
+      Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
+      CoprocessorCallback<R> callback) {
+    return null;
+  }
+
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index 987ac7e..d53353e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -18,10 +18,12 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Optional;
+import java.util.TreeMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
@@ -45,15 +47,14 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
 import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -472,40 +473,17 @@ public class TestReplicaWithCluster {
     final int numRows = 10;
     final byte[] qual = Bytes.toBytes("qual");
     final byte[] val  = Bytes.toBytes("val");
-    final List<Pair<byte[], String>> famPaths = new ArrayList<>();
+    Map<byte[], List<Path>> family2Files = new TreeMap<>(Bytes.BYTES_COMPARATOR);
     for (HColumnDescriptor col : hdt.getColumnFamilies()) {
       Path hfile = new Path(dir, col.getNameAsString());
-      TestHRegionServerBulkLoad.createHFile(HTU.getTestFileSystem(), hfile, col.getName(),
-        qual, val, numRows);
-      famPaths.add(new Pair<>(col.getName(), hfile.toString()));
+      TestHRegionServerBulkLoad.createHFile(HTU.getTestFileSystem(), hfile, col.getName(), qual,
+        val, numRows);
+      family2Files.put(col.getName(), Collections.singletonList(hfile));
     }
 
     // bulk load HFiles
     LOG.debug("Loading test data");
-    final ClusterConnection conn = (ClusterConnection) HTU.getAdmin().getConnection();
-    table = conn.getTable(hdt.getTableName());
-    final String bulkToken =
-        new SecureBulkLoadClient(HTU.getConfiguration(), table).prepareBulkLoad(conn);
-    ClientServiceCallable<Void> callable = new ClientServiceCallable<Void>(conn,
-        hdt.getTableName(), TestHRegionServerBulkLoad.rowkey(0),
-        new RpcControllerFactory(HTU.getConfiguration()).newController(), HConstants.PRIORITY_UNSET) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        LOG.debug("Going to connect to server " + getLocation() + " for row "
-            + Bytes.toStringBinary(getRow()));
-        SecureBulkLoadClient secureClient = null;
-        byte[] regionName = getLocation().getRegionInfo().getRegionName();
-        try (Table table = conn.getTable(getTableName())) {
-          secureClient = new SecureBulkLoadClient(HTU.getConfiguration(), table);
-          secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
-              true, null, bulkToken);
-        }
-        return null;
-      }
-    };
-    RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(HTU.getConfiguration());
-    RpcRetryingCaller<Void> caller = factory.newCaller();
-    caller.callWithRetries(callable, 10000);
+    BulkLoadHFiles.create(HTU.getConfiguration()).bulkLoad(hdt.getTableName(), family2Files);
 
     // verify we can read them from the primary
     LOG.debug("Verifying data load");
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
index b222cc9..e35f0af 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
@@ -72,7 +72,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
@@ -567,7 +567,7 @@ public class TestRegionObserverInterface {
       createHFile(util.getConfiguration(), fs, new Path(familyDir, Bytes.toString(A)), A, A);
 
       // Bulk load
-      new LoadIncrementalHFiles(conf).doBulkLoad(dir, util.getAdmin(), table, locator);
+      BulkLoadHFiles.create(conf).bulkLoad(tableName, dir);
 
       verifyMethodResult(SimpleRegionObserver.class,
         new String[] { "hadPreBulkLoadHFile", "hadPostBulkLoadHFile" }, tableName,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
index 40cd540..84463b3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
@@ -22,41 +22,38 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
 import java.util.Random;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClientServiceCallable;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.collect.HashMultimap;
 import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
 import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
@@ -378,39 +375,21 @@ public class SpaceQuotaHelperForTests {
   /**
    * Bulk-loads a number of files with a number of rows to the given table.
    */
-  ClientServiceCallable<Boolean> generateFileToLoad(
-      TableName tn, int numFiles, int numRowsPerFile) throws Exception {
-    Connection conn = testUtil.getConnection();
+  Map<byte[], List<Path>> generateFileToLoad(TableName tn, int numFiles, int numRowsPerFile)
+      throws Exception {
     FileSystem fs = testUtil.getTestFileSystem();
-    Configuration conf = testUtil.getConfiguration();
     Path baseDir = new Path(fs.getHomeDirectory(), testName.getMethodName() + "_files");
     fs.mkdirs(baseDir);
-    final List<Pair<byte[], String>> famPaths = new ArrayList<>();
+    List<Path> hfiles = new ArrayList<>();
     for (int i = 1; i <= numFiles; i++) {
       Path hfile = new Path(baseDir, "file" + i);
-      TestHRegionServerBulkLoad.createHFile(
-          fs, hfile, Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("my"),
-          Bytes.toBytes("file"), numRowsPerFile);
-      famPaths.add(new Pair<>(Bytes.toBytes(SpaceQuotaHelperForTests.F1), hfile.toString()));
+      TestHRegionServerBulkLoad.createHFile(fs, hfile, Bytes.toBytes(SpaceQuotaHelperForTests.F1),
+        Bytes.toBytes("my"), Bytes.toBytes("file"), numRowsPerFile);
+      hfiles.add(hfile);
     }
-
-    // bulk load HFiles
-    Table table = conn.getTable(tn);
-    final String bulkToken = new SecureBulkLoadClient(conf, table).prepareBulkLoad(conn);
-    return new ClientServiceCallable<Boolean>(
-        conn, tn, Bytes.toBytes("row"), new RpcControllerFactory(conf).newController(),
-        HConstants.PRIORITY_UNSET) {
-      @Override
-     public Boolean rpcCall() throws Exception {
-        SecureBulkLoadClient secureClient = null;
-        byte[] regionName = getLocation().getRegion().getRegionName();
-        try (Table table = conn.getTable(getTableName())) {
-          secureClient = new SecureBulkLoadClient(conf, table);
-          return secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
-                true, null, bulkToken);
-        }
-      }
-    };
+    Map<byte[], List<Path>> family2Files = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    family2Files.put(Bytes.toBytes(SpaceQuotaHelperForTests.F1), hfiles);
+    return family2Files;
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestLowLatencySpaceQuotas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestLowLatencySpaceQuotas.java
index fdc7ad3..9e3dd58 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestLowLatencySpaceQuotas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestLowLatencySpaceQuotas.java
@@ -17,12 +17,13 @@
 package org.apache.hadoop.hbase.quotas;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -31,11 +32,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClientServiceCallable;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.RpcRetryingCaller;
-import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.client.SnapshotType;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaHelperForTests.SpaceQuotaSnapshotPredicate;
@@ -43,6 +41,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -214,7 +213,7 @@ public class TestLowLatencySpaceQuotas {
         tn, SpaceQuotaHelperForTests.ONE_GIGABYTE, SpaceViolationPolicy.NO_INSERTS);
     admin.setQuota(settings);
 
-    ClientServiceCallable<Boolean> callable = helper.generateFileToLoad(tn, 3, 550);
+    Map<byte[], List<Path>> family2Files = helper.generateFileToLoad(tn, 3, 550);
     // Make sure the files are about as long as we expect
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     FileStatus[] files = fs.listStatus(
@@ -228,13 +227,13 @@ public class TestLowLatencySpaceQuotas {
       totalSize += file.getLen();
     }
 
-    RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
-    RpcRetryingCaller<Boolean> caller = factory.<Boolean> newCaller();
-    assertTrue("The bulk load failed", caller.callWithRetries(callable, Integer.MAX_VALUE));
+    assertFalse("The bulk load failed",
+      BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(tn, family2Files).isEmpty());
 
     final long finalTotalSize = totalSize;
     TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, tn) {
-      @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception {
+      @Override
+      boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception {
         return snapshot.getUsage() >= finalTotalSize;
       }
     });
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
index 4b96f3d..c992ac1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
@@ -17,13 +17,17 @@
  */
 package org.apache.hadoop.hbase.quotas;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 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.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.atomic.AtomicLong;
@@ -38,7 +42,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.ClientServiceCallable;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Increment;
@@ -47,8 +50,6 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.RpcRetryingCaller;
-import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -56,6 +57,7 @@ import org.apache.hadoop.hbase.quotas.policies.DefaultViolationPolicyEnforcement
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.AfterClass;
@@ -237,19 +239,18 @@ public class TestSpaceQuotas {
   @Test
   public void testNoBulkLoadsWithNoWrites() throws Exception {
     Put p = new Put(Bytes.toBytes("to_reject"));
-    p.addColumn(
-        Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
+    p.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
+      Bytes.toBytes("reject"));
     TableName tableName = writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, p);
 
     // The table is now in violation. Try to do a bulk load
-    ClientServiceCallable<Boolean> callable = helper.generateFileToLoad(tableName, 1, 50);
-    RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
-    RpcRetryingCaller<Boolean> caller = factory.newCaller();
+    Map<byte[], List<Path>> family2Files = helper.generateFileToLoad(tableName, 1, 50);
     try {
-      caller.callWithRetries(callable, Integer.MAX_VALUE);
+      BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(tableName, family2Files);
       fail("Expected the bulk load call to fail!");
-    } catch (SpaceLimitingException e) {
+    } catch (IOException e) {
       // Pass
+      assertThat(e.getCause(), instanceOf(SpaceLimitingException.class));
       LOG.trace("Caught expected exception", e);
     }
   }
@@ -293,7 +294,7 @@ public class TestSpaceQuotas {
         enforcement instanceof DefaultViolationPolicyEnforcement);
 
     // Should generate two files, each of which is over 25KB each
-    ClientServiceCallable<Boolean> callable = helper.generateFileToLoad(tn, 2, 525);
+    Map<byte[], List<Path>> family2Files = helper.generateFileToLoad(tn, 2, 525);
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     FileStatus[] files = fs.listStatus(
         new Path(fs.getHomeDirectory(), testName.getMethodName() + "_files"));
@@ -305,13 +306,12 @@ public class TestSpaceQuotas {
       LOG.debug(file.getPath() + " -> " + file.getLen() +"B");
     }
 
-    RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
-    RpcRetryingCaller<Boolean> caller = factory.newCaller();
     try {
-      caller.callWithRetries(callable, Integer.MAX_VALUE);
+      BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(tn, family2Files);
       fail("Expected the bulk load call to fail!");
-    } catch (SpaceLimitingException e) {
+    } catch (IOException e) {
       // Pass
+      assertThat(e.getCause(), instanceOf(SpaceLimitingException.class));
       LOG.trace("Caught expected exception", e);
     }
     // Verify that we have no data in the table because neither file should have been
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index c86f3e1..fd02cf4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -25,9 +25,11 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -53,7 +55,6 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.RpcRetryingCaller;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
@@ -71,8 +72,8 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
@@ -204,59 +205,37 @@ public class TestHRegionServerBulkLoad {
       // create HFiles for different column families
       FileSystem fs = UTIL.getTestFileSystem();
       byte[] val = Bytes.toBytes(String.format("%010d", iteration));
-      final List<Pair<byte[], String>> famPaths = new ArrayList<>(NUM_CFS);
+      Map<byte[], List<Path>> family2Files = new TreeMap<>(Bytes.BYTES_COMPARATOR);
       for (int i = 0; i < NUM_CFS; i++) {
         Path hfile = new Path(dir, family(i));
         byte[] fam = Bytes.toBytes(family(i));
         createHFile(fs, hfile, fam, QUAL, val, 1000);
-        famPaths.add(new Pair<>(fam, hfile.toString()));
+        family2Files.put(fam, Collections.singletonList(hfile));
       }
-
       // bulk load HFiles
-      final ClusterConnection conn = (ClusterConnection)UTIL.getConnection();
-      Table table = conn.getTable(tableName);
-      final String bulkToken = new SecureBulkLoadClient(UTIL.getConfiguration(), table).
-          prepareBulkLoad(conn);
-      ClientServiceCallable<Void> callable = new ClientServiceCallable<Void>(conn,
-          tableName, Bytes.toBytes("aaa"),
-          new RpcControllerFactory(UTIL.getConfiguration()).newController(), HConstants.PRIORITY_UNSET) {
-        @Override
-        public Void rpcCall() throws Exception {
-          LOG.debug("Going to connect to server " + getLocation() + " for row "
-              + Bytes.toStringBinary(getRow()));
-          SecureBulkLoadClient secureClient = null;
-          byte[] regionName = getLocation().getRegionInfo().getRegionName();
-          try (Table table = conn.getTable(getTableName())) {
-            secureClient = new SecureBulkLoadClient(UTIL.getConfiguration(), table);
-            secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
-                  true, null, bulkToken);
-          }
-          return null;
-        }
-      };
-      RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(conf);
-      RpcRetryingCaller<Void> caller = factory.<Void> newCaller();
-      caller.callWithRetries(callable, Integer.MAX_VALUE);
-
+      BulkLoadHFiles.create(UTIL.getConfiguration()).bulkLoad(tableName, family2Files);
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
+        RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(conf);
+        RpcRetryingCaller<Void> caller = factory.<Void> newCaller();
         // 5 * 50 = 250 open file handles!
-        callable = new ClientServiceCallable<Void>(conn,
-            tableName, Bytes.toBytes("aaa"),
-            new RpcControllerFactory(UTIL.getConfiguration()).newController(), HConstants.PRIORITY_UNSET) {
-          @Override
-          protected Void rpcCall() throws Exception {
-            LOG.debug("compacting " + getLocation() + " for row "
-                + Bytes.toStringBinary(getRow()));
-            AdminProtos.AdminService.BlockingInterface server =
-              conn.getAdmin(getLocation().getServerName());
-            CompactRegionRequest request = RequestConverter.buildCompactRegionRequest(
+        ClientServiceCallable<Void> callable =
+          new ClientServiceCallable<Void>(UTIL.getConnection(), tableName, Bytes.toBytes("aaa"),
+            new RpcControllerFactory(UTIL.getConfiguration()).newController(),
+            HConstants.PRIORITY_UNSET) {
+            @Override
+            protected Void rpcCall() throws Exception {
+              LOG.debug(
+                "compacting " + getLocation() + " for row " + Bytes.toStringBinary(getRow()));
+              AdminProtos.AdminService.BlockingInterface server =
+                ((ClusterConnection) UTIL.getConnection()).getAdmin(getLocation().getServerName());
+              CompactRegionRequest request = RequestConverter.buildCompactRegionRequest(
                 getLocation().getRegionInfo().getRegionName(), true, null);
-            server.compactRegion(null, request);
-            numCompactions.incrementAndGet();
-            return null;
-          }
-        };
+              server.compactRegion(null, request);
+              numCompactions.incrementAndGet();
+              return null;
+            }
+          };
         caller.callWithRetries(callable, Integer.MAX_VALUE);
       }
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
index f30f084..1fe1f3f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -45,7 +44,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -94,10 +93,7 @@ public class TestScannerWithBulkload {
       false);
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true);
-    final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
-    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
-      bulkload.doBulkLoad(hfilePath, admin, table, locator);
-    }
+    BulkLoadHFiles.create(conf).bulkLoad(tableName, hfilePath);
     ResultScanner scanner = table.getScanner(scan);
     Result result = scanner.next();
     result = scanAfterBulkLoad(scanner, result, "version2");
@@ -233,7 +229,7 @@ public class TestScannerWithBulkload {
         "/temp/testBulkLoadWithParallelScan/col/file", false);
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true);
-    final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
+    final BulkLoadHFiles bulkload = BulkLoadHFiles.create(conf);
     ResultScanner scanner = table.getScanner(scan);
     Result result = scanner.next();
     // Create a scanner and then do bulk load
@@ -246,9 +242,7 @@ public class TestScannerWithBulkload {
           put1.add(new KeyValue(Bytes.toBytes("row5"), Bytes.toBytes("col"), Bytes.toBytes("q"), l,
               Bytes.toBytes("version0")));
           table.put(put1);
-          try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
-            bulkload.doBulkLoad(hfilePath, admin, table, locator);
-          }
+          bulkload.bulkLoad(tableName, hfilePath);
           latch.countDown();
         } catch (TableNotFoundException e) {
         } catch (IOException e) {
@@ -276,10 +270,7 @@ public class TestScannerWithBulkload {
       "/temp/testBulkLoadNativeHFile/col/file", true);
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true);
-    final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
-    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
-      bulkload.doBulkLoad(hfilePath, admin, table, locator);
-    }
+    BulkLoadHFiles.create(conf).bulkLoad(tableName, hfilePath);
     ResultScanner scanner = table.getScanner(scan);
     Result result = scanner.next();
     // We had 'version0', 'version1' for 'row1,col:q' in the table.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
index eb25806..5c73d07 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
@@ -21,10 +21,8 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Deque;
 import java.util.Map;
-import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Consumer;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
@@ -32,8 +30,8 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
@@ -47,7 +45,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
@@ -59,6 +57,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
 
 
@@ -178,7 +177,7 @@ public class TestSecureBulkLoadManager {
 
   /**
    * A trick is used to make sure server-side failures( if any ) not being covered up by a client
-   * retry. Since LoadIncrementalHFiles.doBulkLoad keeps performing bulkload calls as long as the
+   * retry. Since BulkLoadHFilesTool.bulkLoad keeps performing bulkload calls as long as the
    * HFile queue is not empty, while server-side exceptions in the doAs block do not lead
    * to a client exception, a bulkload will always succeed in this case by default, thus client
    * will never be aware that failures have ever happened . To avoid this kind of retry ,
@@ -187,23 +186,23 @@ public class TestSecureBulkLoadManager {
    * once, and server-side failures, if any ,can be checked via data.
    */
   class MyExceptionToAvoidRetry extends DoNotRetryIOException {
+
+    private static final long serialVersionUID = -6802760664998771151L;
   }
 
   private void doBulkloadWithoutRetry(Path dir) throws Exception {
-    Connection connection = testUtil.getConnection();
-    LoadIncrementalHFiles h = new LoadIncrementalHFiles(conf) {
+    BulkLoadHFilesTool h = new BulkLoadHFilesTool(conf) {
+
       @Override
-      protected void bulkLoadPhase(final Table htable, final Connection conn,
-          ExecutorService pool, Deque<LoadQueueItem> queue,
-          final Multimap<ByteBuffer, LoadQueueItem> regionGroups, boolean copyFile,
-          Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
-        super.bulkLoadPhase(htable, conn, pool, queue, regionGroups, copyFile, item2RegionMap);
+      protected void bulkLoadPhase(AsyncClusterConnection conn, TableName tableName,
+          Deque<LoadQueueItem> queue, Multimap<ByteBuffer, LoadQueueItem> regionGroups,
+          boolean copyFiles, Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
+        super.bulkLoadPhase(conn, tableName, queue, regionGroups, copyFiles, item2RegionMap);
         throw new MyExceptionToAvoidRetry(); // throw exception to avoid retry
       }
     };
     try {
-      h.doBulkLoad(dir, testUtil.getAdmin(), connection.getTable(TABLE),
-          connection.getRegionLocator(TABLE));
+      h.bulkLoad(TABLE, dir);
       Assert.fail("MyExceptionToAvoidRetry is expected");
     } catch (MyExceptionToAvoidRetry e) { //expected
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
index 37ca7dc..f546058 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
@@ -67,7 +67,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.replication.regionserver.TestSourceFSConfigurationProvider;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -622,9 +622,7 @@ public class TestMasterReplication {
 
     Table source = tables[masterNumber];
     final TableName tableName = source.getName();
-    LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration());
-    String[] args = { dir.toString(), tableName.toString() };
-    loader.run(args);
+    BulkLoadHFiles.create(util.getConfiguration()).bulkLoad(tableName, dir);
 
     if (toValidate) {
       for (int slaveClusterNumber : slaveNumbers) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
index 2d6c28f..eb3a7a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
@@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
+import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -275,7 +275,7 @@ public class TestReplicationSink {
   }
 
   @Test
-  public void testRethrowRetriesExhaustedWithDetailsException() throws Exception {
+  public void testRethrowRetriesExhaustedException() throws Exception {
     TableName notExistTable = TableName.valueOf("notExistTable");
     List<WALEntry> entries = new ArrayList<>();
     List<Cell> cells = new ArrayList<>();
@@ -300,7 +300,7 @@ public class TestReplicationSink {
           SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
             replicationClusterId, baseNamespaceDir, hfileArchiveDir);
           Assert.fail("Should re-throw RetriesExhaustedWithDetailsException.");
-        } catch (RetriesExhaustedWithDetailsException e) {
+        } catch (RetriesExhaustedException e) {
         } finally {
           admin.enableTable(TABLE_NAME1);
         }
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 aa62470..41d4f46 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
@@ -21,48 +21,31 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.net.SocketAddress;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellBuilder;
 import org.apache.hadoop.hbase.CellBuilderFactory;
 import org.apache.hadoop.hbase.CellBuilderType;
 import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.BufferedMutator;
-import org.apache.hadoop.hbase.client.BufferedMutatorParams;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-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.RegionLocator;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+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.Row;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableBuilder;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.coprocessor.Batch;
-import org.apache.hadoop.hbase.filter.CompareFilter;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -82,15 +65,16 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 /**
  * Simple test of sink-side wal entry filter facility.
  */
-@Category({ReplicationTests.class, SmallTests.class})
+@Category({ ReplicationTests.class, SmallTests.class })
 public class TestWALEntrySinkFilter {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestWALEntrySinkFilter.class);
+    HBaseClassTestRule.forClass(TestWALEntrySinkFilter.class);
 
   private static final Logger LOG = LoggerFactory.getLogger(TestReplicationSink.class);
-  @Rule public TestName name = new TestName();
+  @Rule
+  public TestName name = new TestName();
   static final int BOUNDARY = 5;
   static final AtomicInteger UNFILTERED = new AtomicInteger();
   static final AtomicInteger FILTERED = new AtomicInteger();
@@ -114,55 +98,48 @@ public class TestWALEntrySinkFilter {
   };
 
   /**
-   * Test filter.
-   * Filter will filter out any write time that is <= 5 (BOUNDARY). We count how many items we
-   * filter out and we count how many cells make it through for distribution way down below in the
-   * Table#batch implementation. Puts in place a custom DevNullConnection so we can insert our
-   * counting Table.
+   * Test filter. Filter will filter out any write time that is <= 5 (BOUNDARY). We count how many
+   * items we filter out and we count how many cells make it through for distribution way down below
+   * in the Table#batch implementation. Puts in place a custom DevNullConnection so we can insert
+   * our counting Table.
    * @throws IOException
    */
   @Test
   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(WALEntrySinkFilter.WAL_ENTRY_FILTER_KEY,
-        IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl.class, WALEntrySinkFilter.class);
-    conf.setClass("hbase.client.connection.impl", DevNullConnection.class,
-      Connection.class);
+      IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl.class, WALEntrySinkFilter.class);
+    conf.setClass(ClusterConnectionFactory.HBASE_SERVER_CLUSTER_CONNECTION_IMPL,
+      DevNullAsyncClusterConnection.class, AsyncClusterConnection.class);
     ReplicationSink sink = new ReplicationSink(conf, STOPPABLE);
     // Create some dumb walentries.
-    List< org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry > entries =
-        new ArrayList<>();
+    List<AdminProtos.WALEntry> entries = new ArrayList<>();
     AdminProtos.WALEntry.Builder entryBuilder = AdminProtos.WALEntry.newBuilder();
     // Need a tablename.
     ByteString tableName =
-        ByteString.copyFromUtf8(TableName.valueOf(this.name.getMethodName()).toString());
+      ByteString.copyFromUtf8(TableName.valueOf(this.name.getMethodName()).toString());
     // Add WALEdit Cells to Cells List. The way edits arrive at the sink is with protos
     // describing the edit with all Cells from all edits aggregated in a single CellScanner.
     final List<Cell> cells = new ArrayList<>();
     int count = BOUNDARY * 2;
-    for(int i = 0; i < count; i++) {
-      byte [] bytes = Bytes.toBytes(i);
+    for (int i = 0; i < count; i++) {
+      byte[] bytes = Bytes.toBytes(i);
       // Create a wal entry. Everything is set to the current index as bytes or int/long.
       entryBuilder.clear();
-      entryBuilder.setKey(entryBuilder.getKeyBuilder().
-          setLogSequenceNumber(i).
-          setEncodedRegionName(ByteString.copyFrom(bytes)).
-          setWriteTime(i).
-          setTableName(tableName).build());
+      entryBuilder.setKey(entryBuilder.getKeyBuilder().setLogSequenceNumber(i)
+        .setEncodedRegionName(ByteString.copyFrom(bytes)).setWriteTime(i).setTableName(tableName)
+        .build());
       // Lets have one Cell associated with each WALEdit.
       entryBuilder.setAssociatedCellCount(1);
       entries.add(entryBuilder.build());
       // We need to add a Cell per WALEdit to the cells array.
       CellBuilder cellBuilder = CellBuilderFactory.create(CellBuilderType.DEEP_COPY);
       // Make cells whose row, family, cell, value, and ts are == 'i'.
-      Cell cell = cellBuilder.
-          setRow(bytes).
-          setFamily(bytes).
-          setQualifier(bytes).
-          setType(Cell.Type.Put).
-          setTimestamp(i).
-          setValue(bytes).build();
+      Cell cell = cellBuilder.setRow(bytes).setFamily(bytes).setQualifier(bytes)
+        .setType(Cell.Type.Put).setTimestamp(i).setValue(bytes).build();
       cells.add(cell);
     }
     // Now wrap our cells array in a CellScanner that we can pass in to replicateEntries. It has
@@ -193,11 +170,13 @@ public class TestWALEntrySinkFilter {
   /**
    * Simple filter that will filter out any entry wholse writeTime is <= 5.
    */
-  public static class IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl implements WALEntrySinkFilter {
-    public IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl() {}
+  public static class IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl
+      implements WALEntrySinkFilter {
+    public IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl() {
+    }
 
     @Override
-    public void init(Connection connection) {
+    public void init(AsyncConnection conn) {
       // Do nothing.
     }
 
@@ -211,346 +190,48 @@ public class TestWALEntrySinkFilter {
     }
   }
 
-  /**
-   * A DevNull Connection whose only purpose is checking what edits made it through. See down in
-   * {@link Table#batch(List, Object[])}.
-   */
-  public static class DevNullConnection implements Connection {
-    private final Configuration configuration;
-
-    DevNullConnection(Configuration configuration, ExecutorService es, User user) {
-      this.configuration = configuration;
-    }
-
-    @Override
-    public void abort(String why, Throwable e) {
-
-    }
-
-    @Override
-    public boolean isAborted() {
-      return false;
-    }
-
-    @Override
-    public Configuration getConfiguration() {
-      return this.configuration;
-    }
+  public static class DevNullAsyncRegistry extends DummyAsyncRegistry {
 
-    @Override
-    public BufferedMutator getBufferedMutator(TableName tableName) throws IOException {
-      return null;
+    public DevNullAsyncRegistry(Configuration conf) {
     }
 
     @Override
-    public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException {
-      return null;
-    }
-
-    @Override
-    public RegionLocator getRegionLocator(TableName tableName) throws IOException {
-      return null;
-    }
-
-    @Override
-    public Admin getAdmin() throws IOException {
-      return null;
+    public CompletableFuture<String> getClusterId() {
+      return CompletableFuture.completedFuture("test");
     }
+  }
 
-    @Override
-    public void close() throws IOException {
+  public static class DevNullAsyncClusterConnection extends DummyAsyncClusterConnection {
 
-    }
+    private final Configuration conf;
 
-    @Override
-    public boolean isClosed() {
-      return false;
+    public DevNullAsyncClusterConnection(Configuration conf, Object registry, String clusterId,
+        SocketAddress localAddress, User user) {
+      this.conf = conf;
     }
 
     @Override
-    public TableBuilder getTableBuilder(final TableName tableName, ExecutorService pool) {
-      return new TableBuilder() {
-        @Override
-        public TableBuilder setOperationTimeout(int timeout) {
-          return this;
-        }
-
-        @Override
-        public TableBuilder setRpcTimeout(int timeout) {
-          return this;
-        }
+    public AsyncTable<AdvancedScanResultConsumer> getTable(TableName tableName) {
+      return new DummyAsyncTable<AdvancedScanResultConsumer>() {
 
         @Override
-        public TableBuilder setReadRpcTimeout(int timeout) {
-          return this;
-        }
-
-        @Override
-        public TableBuilder setWriteRpcTimeout(int timeout) {
-          return this;
-        }
-
-        @Override
-        public Table build() {
-          return new Table() {
-            @Override
-            public TableName getName() {
-              return tableName;
-            }
-
-            @Override
-            public Configuration getConfiguration() {
-              return configuration;
-            }
-
-            @Override
-            public HTableDescriptor getTableDescriptor() throws IOException {
-              return null;
-            }
-
-            @Override
-            public TableDescriptor getDescriptor() throws IOException {
-              return null;
-            }
-
-            @Override
-            public boolean exists(Get get) throws IOException {
-              return false;
-            }
-
-            @Override
-            public boolean[] exists(List<Get> gets) throws IOException {
-              return new boolean[0];
-            }
-
-            @Override
-            public void batch(List<? extends Row> actions, Object[] results) throws IOException, InterruptedException {
-              for (Row action: actions) {
-                // Row is the index of the loop above where we make WALEntry and Cells.
-                int row = Bytes.toInt(action.getRow());
-                assertTrue("" + row, row> BOUNDARY);
-                UNFILTERED.incrementAndGet();
-              }
-            }
-
-            @Override
-            public <R> void batchCallback(List<? extends Row> actions, Object[] results, Batch.Callback<R> callback) throws IOException, InterruptedException {
-
-            }
-
-            @Override
-            public Result get(Get get) throws IOException {
-              return null;
-            }
-
-            @Override
-            public Result[] get(List<Get> gets) throws IOException {
-              return new Result[0];
-            }
-
-            @Override
-            public ResultScanner getScanner(Scan scan) throws IOException {
-              return null;
-            }
-
-            @Override
-            public ResultScanner getScanner(byte[] family) throws IOException {
-              return null;
-            }
-
-            @Override
-            public ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException {
-              return null;
-            }
-
-            @Override
-            public void put(Put put) throws IOException {
-
-            }
-
-            @Override
-            public void put(List<Put> puts) throws IOException {
-
-            }
-
-            @Override
-            public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, byte[] value, Put put) throws IOException {
-              return false;
-            }
-
-            @Override
-            public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, byte[] value, Put put) throws IOException {
-              return false;
-            }
-
-            @Override
-            public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, Put put) throws IOException {
-              return false;
-            }
-
-            @Override
-            public void delete(Delete delete) throws IOException {
-
-            }
-
-            @Override
-            public void delete(List<Delete> deletes) throws IOException {
-
-            }
-
-            @Override
-            public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, byte[] value, Delete delete) throws IOException {
-              return false;
-            }
-
-            @Override
-            public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, byte[] value, Delete delete) throws IOException {
-              return false;
-            }
-
-            @Override
-            public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, Delete delete) throws IOException {
-              return false;
-            }
-
-            @Override
-            public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
-              return null;
-            }
-
-            @Override
-            public void mutateRow(RowMutations rm) throws IOException {
-
-            }
-
-            @Override
-            public Result append(Append append) throws IOException {
-              return null;
-            }
-
-            @Override
-            public Result increment(Increment increment) throws IOException {
-              return null;
-            }
-
-            @Override
-            public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) throws IOException {
-              return 0;
-            }
-
-            @Override
-            public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, Durability durability) throws IOException {
-              return 0;
-            }
-
-            @Override
-            public void close() throws IOException {
-
-            }
-
-            @Override
-            public CoprocessorRpcChannel coprocessorService(byte[] row) {
-              return null;
-            }
-
-            @Override
-            public <T extends com.google.protobuf.Service, R> Map<byte[], R> coprocessorService(Class<T> service, byte[] startKey, byte[] endKey, Batch.Call<T, R> callable) throws com.google.protobuf.ServiceException, Throwable {
-              return null;
-            }
-
-            @Override
-            public <T extends com.google.protobuf.Service, R> void coprocessorService(Class<T> service, byte[] startKey, byte[] endKey, Batch.Call<T, R> callable, Batch.Callback<R> callback) throws com.google.protobuf.ServiceException, Throwable {
-
-            }
-
-            @Override
-            public <R extends com.google.protobuf.Message> Map<byte[], R> batchCoprocessorService(com.google.protobuf.Descriptors.MethodDescriptor methodDescriptor, com.google.protobuf.Message request, byte[] startKey, byte[] endKey, R responsePrototype) throws com.google.protobuf.ServiceException, Throwable {
-              return null;
-            }
-
-            @Override
-            public <R extends com.google.protobuf.Message> void batchCoprocessorService(com.google.protobuf.Descriptors.MethodDescriptor methodDescriptor, com.google.protobuf.Message request, byte[] startKey, byte[] endKey, R responsePrototype, Batch.Callback<R> callback) throws com.google.protobuf.ServiceException, Throwable {
-
-            }
-
-            @Override
-            public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, byte[] value, RowMutations mutation) throws IOException {
-              return false;
-            }
-
-            @Override
-            public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, RowMutations mutation) throws IOException {
-              return false;
-            }
-
-            @Override
-            public long getRpcTimeout(TimeUnit unit) {
-              return 0;
-            }
-
-            @Override
-            public int getRpcTimeout() {
-              return 0;
-            }
-
-            @Override
-            public void setRpcTimeout(int rpcTimeout) {
-
-            }
-
-            @Override
-            public long getReadRpcTimeout(TimeUnit unit) {
-              return 0;
-            }
-
-            @Override
-            public int getReadRpcTimeout() {
-              return 0;
-            }
-
-            @Override
-            public void setReadRpcTimeout(int readRpcTimeout) {
-
-            }
-
-            @Override
-            public long getWriteRpcTimeout(TimeUnit unit) {
-              return 0;
-            }
-
-            @Override
-            public int getWriteRpcTimeout() {
-              return 0;
-            }
-
-            @Override
-            public void setWriteRpcTimeout(int writeRpcTimeout) {
-
-            }
-
-            @Override
-            public long getOperationTimeout(TimeUnit unit) {
-              return 0;
-            }
-
-            @Override
-            public int getOperationTimeout() {
-              return 0;
-            }
-
-            @Override
-            public void setOperationTimeout(int operationTimeout) {
-
-            }
-          };
+        public <T> CompletableFuture<List<T>> batchAll(List<? extends Row> actions) {
+          List<T> list = new ArrayList<>(actions.size());
+          for (Row action : actions) {
+            // Row is the index of the loop above where we make WALEntry and Cells.
+            int row = Bytes.toInt(action.getRow());
+            assertTrue("" + row, row > BOUNDARY);
+            UNFILTERED.incrementAndGet();
+            list.add(null);
+          }
+          return CompletableFuture.completedFuture(list);
         }
       };
     }
 
     @Override
-    public void clearRegionLocationCache() {
+    public Configuration getConfiguration() {
+      return conf;
     }
   }
 }
-
-
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index cee34af..4befa4f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -124,7 +124,7 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Threads;
@@ -1115,14 +1115,8 @@ public class TestAccessController extends SecureTestUtil {
     }
 
     private void bulkLoadHFile(TableName tableName) throws Exception {
-      try (Connection conn = ConnectionFactory.createConnection(conf);
-          Admin admin = conn.getAdmin();
-          RegionLocator locator = conn.getRegionLocator(tableName);
-          Table table = conn.getTable(tableName)) {
-        TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
-        LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
-        loader.doBulkLoad(loadPath, admin, table, locator);
-      }
+      TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
+      BulkLoadHFiles.create(conf).bulkLoad(tableName, loadPath);
     }
 
     private static void setPermission(FileSystem fs, Path dir, FsPermission perm)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFiles.java
similarity index 83%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFiles.java
index 7c04edc..e85fc1a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFiles.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.tool;
 
+import static org.apache.hadoop.hbase.HBaseTestingUtility.countRows;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -72,11 +73,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
  * faster than the full MR cluster tests in TestHFileOutputFormat
  */
 @Category({ MiscTests.class, LargeTests.class })
-public class TestLoadIncrementalHFiles {
+public class TestBulkLoadHFiles {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestLoadIncrementalHFiles.class);
+    HBaseClassTestRule.forClass(TestBulkLoadHFiles.class);
 
   @Rule
   public TestName tn = new TestName();
@@ -89,14 +90,14 @@ public class TestLoadIncrementalHFiles {
   static final int MAX_FILES_PER_REGION_PER_FAMILY = 4;
 
   private static final byte[][] SPLIT_KEYS =
-      new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ppp") };
+    new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ppp") };
 
   static HBaseTestingUtility util = new HBaseTestingUtility();
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
-    util.getConfiguration().setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY,
+    util.getConfiguration().setInt(BulkLoadHFiles.MAX_FILES_PER_REGION_PER_FAMILY,
       MAX_FILES_PER_REGION_PER_FAMILY);
     // change default behavior so that tag values are returned with normal rpcs
     util.getConfiguration().set(HConstants.RPC_CODEC_CONF_KEY,
@@ -119,7 +120,7 @@ public class TestLoadIncrementalHFiles {
   public void testSimpleLoadWithMap() throws Exception {
     runTest("testSimpleLoadWithMap", BloomType.NONE,
       new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
-          new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, },
+        new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, },
       true);
   }
 
@@ -130,16 +131,16 @@ public class TestLoadIncrementalHFiles {
   public void testSimpleLoad() throws Exception {
     runTest("testSimpleLoad", BloomType.NONE,
       new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
-          new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, });
+        new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, });
   }
 
   @Test
   public void testSimpleLoadWithFileCopy() throws Exception {
     String testName = tn.getMethodName();
     final byte[] TABLE_NAME = Bytes.toBytes("mytable_" + testName);
-    runTest(testName, buildHTD(TableName.valueOf(TABLE_NAME), BloomType.NONE),
-        false, null, new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
-          new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, },
+    runTest(testName, buildHTD(TableName.valueOf(TABLE_NAME), BloomType.NONE), false, null,
+      new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
+        new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, },
       false, true, 2);
   }
 
@@ -150,7 +151,7 @@ public class TestLoadIncrementalHFiles {
   public void testRegionCrossingLoad() throws Exception {
     runTest("testRegionCrossingLoad", BloomType.NONE,
       new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("eee") },
-          new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
+        new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
   }
 
   /**
@@ -160,7 +161,7 @@ public class TestLoadIncrementalHFiles {
   public void testRegionCrossingRowBloom() throws Exception {
     runTest("testRegionCrossingLoadRowBloom", BloomType.ROW,
       new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("eee") },
-          new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
+        new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
   }
 
   /**
@@ -170,7 +171,7 @@ public class TestLoadIncrementalHFiles {
   public void testRegionCrossingRowColBloom() throws Exception {
     runTest("testRegionCrossingLoadRowColBloom", BloomType.ROWCOL,
       new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("eee") },
-          new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
+        new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
   }
 
   /**
@@ -181,9 +182,9 @@ public class TestLoadIncrementalHFiles {
   public void testSimpleHFileSplit() throws Exception {
     runTest("testHFileSplit", BloomType.NONE,
       new byte[][] { Bytes.toBytes("aaa"), Bytes.toBytes("fff"), Bytes.toBytes("jjj"),
-          Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"), },
+        Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"), },
       new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("lll") },
-          new byte[][] { Bytes.toBytes("mmm"), Bytes.toBytes("zzz") }, });
+        new byte[][] { Bytes.toBytes("mmm"), Bytes.toBytes("zzz") }, });
   }
 
   /**
@@ -217,27 +218,27 @@ public class TestLoadIncrementalHFiles {
   public void testSplitALot() throws Exception {
     runTest("testSplitALot", BloomType.NONE,
       new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("bbb"), Bytes.toBytes("ccc"),
-          Bytes.toBytes("ddd"), Bytes.toBytes("eee"), Bytes.toBytes("fff"), Bytes.toBytes("ggg"),
-          Bytes.toBytes("hhh"), Bytes.toBytes("iii"), Bytes.toBytes("lll"), Bytes.toBytes("mmm"),
-          Bytes.toBytes("nnn"), Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
-          Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"), Bytes.toBytes("uuu"),
-          Bytes.toBytes("vvv"), Bytes.toBytes("zzz"), },
+        Bytes.toBytes("ddd"), Bytes.toBytes("eee"), Bytes.toBytes("fff"), Bytes.toBytes("ggg"),
+        Bytes.toBytes("hhh"), Bytes.toBytes("iii"), Bytes.toBytes("lll"), Bytes.toBytes("mmm"),
+        Bytes.toBytes("nnn"), Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
+        Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"), Bytes.toBytes("uuu"),
+        Bytes.toBytes("vvv"), Bytes.toBytes("zzz"), },
       new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("zzz") }, });
   }
 
   private void testRegionCrossingHFileSplit(BloomType bloomType) throws Exception {
     runTest("testHFileSplit" + bloomType + "Bloom", bloomType,
       new byte[][] { Bytes.toBytes("aaa"), Bytes.toBytes("fff"), Bytes.toBytes("jjj"),
-          Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"), },
+        Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"), },
       new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("eee") },
-          new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
+        new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
   }
 
   private TableDescriptor buildHTD(TableName tableName, BloomType bloomType) {
     return TableDescriptorBuilder.newBuilder(tableName)
-        .setColumnFamily(
-          ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setBloomFilterType(bloomType).build())
-        .build();
+      .setColumnFamily(
+        ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setBloomFilterType(bloomType).build())
+      .build();
   }
 
   private void runTest(String testName, BloomType bloomType, byte[][][] hfileRanges)
@@ -265,28 +266,24 @@ public class TestLoadIncrementalHFiles {
     runTest(testName, TABLE_WITHOUT_NS, bloomType, preCreateTable, tableSplitKeys, hfileRanges,
       useMap, 2);
 
-
-    /* Run the test bulkloading the table from a depth of 3
-      directory structure is now
-      baseDirectory
-          -- regionDir
-            -- familyDir
-              -- storeFileDir
-    */
+    /*
+     * Run the test bulkloading the table from a depth of 3 directory structure is now baseDirectory
+     * -- regionDir -- familyDir -- storeFileDir
+     */
     if (preCreateTable) {
-      runTest(testName + 2, TABLE_WITHOUT_NS, bloomType, true, tableSplitKeys, hfileRanges,
-          false, 3);
+      runTest(testName + 2, TABLE_WITHOUT_NS, bloomType, true, tableSplitKeys, hfileRanges, false,
+        3);
     }
 
     // Run the test bulkloading the table to the specified namespace
     final TableName TABLE_WITH_NS = TableName.valueOf(Bytes.toBytes(NAMESPACE), TABLE_NAME);
-    runTest(testName, TABLE_WITH_NS, bloomType, preCreateTable, tableSplitKeys, hfileRanges,
-      useMap, 2);
+    runTest(testName, TABLE_WITH_NS, bloomType, preCreateTable, tableSplitKeys, hfileRanges, useMap,
+      2);
   }
 
   private void runTest(String testName, TableName tableName, BloomType bloomType,
-      boolean preCreateTable, byte[][] tableSplitKeys, byte[][][] hfileRanges,
-      boolean useMap, int depth) throws Exception {
+      boolean preCreateTable, byte[][] tableSplitKeys, byte[][][] hfileRanges, boolean useMap,
+      int depth) throws Exception {
     TableDescriptor htd = buildHTD(tableName, bloomType);
     runTest(testName, htd, preCreateTable, tableSplitKeys, hfileRanges, useMap, false, depth);
   }
@@ -296,7 +293,7 @@ public class TestLoadIncrementalHFiles {
       byte[][][] hfileRanges, boolean useMap, boolean deleteFile, boolean copyFiles,
       int initRowCount, int factor) throws Exception {
     return loadHFiles(testName, htd, util, fam, qual, preCreateTable, tableSplitKeys, hfileRanges,
-        useMap, deleteFile, copyFiles, initRowCount, factor, 2);
+      useMap, deleteFile, copyFiles, initRowCount, factor, 2);
   }
 
   public static int loadHFiles(String testName, TableDescriptor htd, HBaseTestingUtility util,
@@ -343,7 +340,7 @@ public class TestLoadIncrementalHFiles {
 
     Configuration conf = util.getConfiguration();
     if (copyFiles) {
-      conf.setBoolean(LoadIncrementalHFiles.ALWAYS_COPY_FILES, true);
+      conf.setBoolean(BulkLoadHFiles.ALWAYS_COPY_FILES, true);
     }
     BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf);
     List<String> args = Lists.newArrayList(baseDirectory.toString(), tableName.toString());
@@ -374,26 +371,23 @@ public class TestLoadIncrementalHFiles {
       }
     }
 
-    Table table = util.getConnection().getTable(tableName);
-    try {
-      assertEquals(initRowCount + expectedRows, util.countRows(table));
-    } finally {
-      table.close();
+    try (Table table = util.getConnection().getTable(tableName)) {
+      assertEquals(initRowCount + expectedRows, countRows(table));
     }
 
     return expectedRows;
   }
 
-  private void runTest(String testName, TableDescriptor htd,
-      boolean preCreateTable, byte[][] tableSplitKeys, byte[][][] hfileRanges, boolean useMap,
-      boolean copyFiles, int depth) throws Exception {
+  private void runTest(String testName, TableDescriptor htd, boolean preCreateTable,
+      byte[][] tableSplitKeys, byte[][][] hfileRanges, boolean useMap, boolean copyFiles, int depth)
+      throws Exception {
     loadHFiles(testName, htd, util, FAMILY, QUALIFIER, preCreateTable, tableSplitKeys, hfileRanges,
       useMap, true, copyFiles, 0, 1000, depth);
 
     final TableName tableName = htd.getTableName();
     // verify staging folder has been cleaned up
     Path stagingBasePath =
-        new Path(FSUtils.getRootDir(util.getConfiguration()), HConstants.BULKLOAD_STAGING_DIR_NAME);
+      new Path(FSUtils.getRootDir(util.getConfiguration()), HConstants.BULKLOAD_STAGING_DIR_NAME);
     FileSystem fs = util.getTestFileSystem();
     if (fs.exists(stagingBasePath)) {
       FileStatus[] files = fs.listStatus(stagingBasePath);
@@ -419,7 +413,7 @@ public class TestLoadIncrementalHFiles {
     Path familyDir = new Path(dir, Bytes.toString(FAMILY));
     // table has these split points
     byte[][] tableSplitKeys = new byte[][] { Bytes.toBytes("aaa"), Bytes.toBytes("fff"),
-        Bytes.toBytes("jjj"), Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"), };
+      Bytes.toBytes("jjj"), Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"), };
 
     // creating an hfile that has values that span the split points.
     byte[] from = Bytes.toBytes("ddd");
@@ -432,13 +426,11 @@ public class TestLoadIncrementalHFiles {
     TableDescriptor htd = buildHTD(tableName, BloomType.NONE);
     util.getAdmin().createTable(htd, tableSplitKeys);
 
-    LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration());
-    String[] args = { dir.toString(), tableName.toString() };
-    loader.run(args);
+    BulkLoadHFiles.create(util.getConfiguration()).bulkLoad(tableName, dir);
 
     Table table = util.getConnection().getTable(tableName);
     try {
-      assertEquals(expectedRows, util.countRows(table));
+      assertEquals(expectedRows, countRows(table));
       HFileTestUtil.verifyTags(table);
     } finally {
       table.close();
@@ -454,16 +446,16 @@ public class TestLoadIncrementalHFiles {
   public void testNonexistentColumnFamilyLoad() throws Exception {
     String testName = tn.getMethodName();
     byte[][][] hFileRanges =
-        new byte[][][] { new byte[][] { Bytes.toBytes("aaa"), Bytes.toBytes("ccc") },
-            new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, };
+      new byte[][][] { new byte[][] { Bytes.toBytes("aaa"), Bytes.toBytes("ccc") },
+        new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, };
 
     byte[] TABLE = Bytes.toBytes("mytable_" + testName);
     // set real family name to upper case in purpose to simulate the case that
     // family name in HFiles is invalid
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(TABLE))
-        .setColumnFamily(ColumnFamilyDescriptorBuilder
-            .of(Bytes.toBytes(new String(FAMILY).toUpperCase(Locale.ROOT))))
-        .build();
+      .setColumnFamily(ColumnFamilyDescriptorBuilder
+        .of(Bytes.toBytes(new String(FAMILY).toUpperCase(Locale.ROOT))))
+      .build();
 
     try {
       runTest(testName, htd, true, SPLIT_KEYS, hFileRanges, false, false, 2);
@@ -474,7 +466,7 @@ public class TestLoadIncrementalHFiles {
       String errMsg = e.getMessage();
       assertTrue(
         "Incorrect exception message, expected message: [" + EXPECTED_MSG_FOR_NON_EXISTING_FAMILY +
-            "], current message: [" + errMsg + "]",
+          "], current message: [" + errMsg + "]",
         errMsg.contains(EXPECTED_MSG_FOR_NON_EXISTING_FAMILY));
     }
   }
@@ -517,10 +509,8 @@ public class TestLoadIncrementalHFiles {
       } else {
         table = util.getConnection().getTable(TableName.valueOf(tableName));
       }
-
-      final String[] args = { dir.toString(), tableName };
-      new LoadIncrementalHFiles(util.getConfiguration()).run(args);
-      assertEquals(500, util.countRows(table));
+      BulkLoadHFiles.create(util.getConfiguration()).bulkLoad(TableName.valueOf(tableName), dir);
+      assertEquals(500, countRows(table));
     } finally {
       if (table != null) {
         table.close();
@@ -560,7 +550,7 @@ public class TestLoadIncrementalHFiles {
     Path bottomOut = new Path(dir, "bottom.out");
     Path topOut = new Path(dir, "top.out");
 
-    LoadIncrementalHFiles.splitStoreFile(util.getConfiguration(), testIn, familyDesc,
+    BulkLoadHFilesTool.splitStoreFile(util.getConfiguration(), testIn, familyDesc,
       Bytes.toBytes("ggg"), bottomOut, topOut);
 
     int rowCount = verifyHFile(bottomOut);
@@ -594,14 +584,14 @@ public class TestLoadIncrementalHFiles {
     FileSystem fs = util.getTestFileSystem();
     Path testIn = new Path(dir, "testhfile");
     ColumnFamilyDescriptor familyDesc =
-        ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setDataBlockEncoding(cfEncoding).build();
+      ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setDataBlockEncoding(cfEncoding).build();
     HFileTestUtil.createHFileWithDataBlockEncoding(util.getConfiguration(), fs, testIn,
       bulkloadEncoding, FAMILY, QUALIFIER, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), 1000);
 
     Path bottomOut = new Path(dir, "bottom.out");
     Path topOut = new Path(dir, "top.out");
 
-    LoadIncrementalHFiles.splitStoreFile(util.getConfiguration(), testIn, familyDesc,
+    BulkLoadHFilesTool.splitStoreFile(util.getConfiguration(), testIn, familyDesc,
       Bytes.toBytes("ggg"), bottomOut, topOut);
 
     int rowCount = verifyHFile(bottomOut);
@@ -612,7 +602,7 @@ public class TestLoadIncrementalHFiles {
   private int verifyHFile(Path p) throws IOException {
     Configuration conf = util.getConfiguration();
     HFile.Reader reader =
-        HFile.createReader(p.getFileSystem(conf), p, new CacheConfig(conf), true, conf);
+      HFile.createReader(p.getFileSystem(conf), p, new CacheConfig(conf), true, conf);
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, false);
     scanner.seekTo();
@@ -682,7 +672,7 @@ public class TestLoadIncrementalHFiles {
     last = "w";
     addStartEndKeysForTest(map, Bytes.toBytes(first), Bytes.toBytes(last));
 
-    byte[][] keysArray = LoadIncrementalHFiles.inferBoundaries(map);
+    byte[][] keysArray = BulkLoadHFilesTool.inferBoundaries(map);
     byte[][] compare = new byte[3][];
     compare[0] = Bytes.toBytes("m");
     compare[1] = Bytes.toBytes("r");
@@ -709,22 +699,21 @@ public class TestLoadIncrementalHFiles {
         FAMILY, QUALIFIER, from, to, 1000);
     }
 
-    LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration());
-    String[] args = { dir.toString(), "mytable_testLoadTooMayHFiles" };
     try {
-      loader.run(args);
+      BulkLoadHFiles.create(util.getConfiguration())
+        .bulkLoad(TableName.valueOf("mytable_testLoadTooMayHFiles"), dir);
       fail("Bulk loading too many files should fail");
     } catch (IOException ie) {
       assertTrue(ie.getMessage()
-          .contains("Trying to load more than " + MAX_FILES_PER_REGION_PER_FAMILY + " hfiles"));
+        .contains("Trying to load more than " + MAX_FILES_PER_REGION_PER_FAMILY + " hfiles"));
     }
   }
 
   @Test(expected = TableNotFoundException.class)
   public void testWithoutAnExistingTableAndCreateTableSetToNo() throws Exception {
     Configuration conf = util.getConfiguration();
-    conf.set(LoadIncrementalHFiles.CREATE_TABLE_CONF_KEY, "no");
-    LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
+    conf.set(BulkLoadHFiles.CREATE_TABLE_CONF_KEY, "no");
+    BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf);
     String[] args = { "directory", "nonExistingTable" };
     loader.run(args);
   }
@@ -741,19 +730,11 @@ public class TestLoadIncrementalHFiles {
     byte[] to = Bytes.toBytes("end");
     Configuration conf = util.getConfiguration();
     String tableName = tn.getMethodName();
-    Table table = util.createTable(TableName.valueOf(tableName), family);
-    HFileTestUtil.createHFile(conf, fs, new Path(familyDir, "hfile"), Bytes.toBytes(family),
-      QUALIFIER, from, to, 1000);
-
-    LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
-    String[] args = { dir.toString(), tableName };
-    try {
-      loader.run(args);
-      assertEquals(1000, util.countRows(table));
-    } finally {
-      if (null != table) {
-        table.close();
-      }
+    try (Table table = util.createTable(TableName.valueOf(tableName), family)) {
+      HFileTestUtil.createHFile(conf, fs, new Path(familyDir, "hfile"), Bytes.toBytes(family),
+        QUALIFIER, from, to, 1000);
+      BulkLoadHFiles.create(conf).bulkLoad(table.getName(), dir);
+      assertEquals(1000, countRows(table));
     }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesSplitRecovery.java
new file mode 100644
index 0000000..2aef16e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesSplitRecovery.java
@@ -0,0 +1,486 @@
+/**
+ * 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.tool;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.spy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Deque;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.IntStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.AfterClass;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
+/**
+ * Test cases for the atomic load error handling of the bulk load functionality.
+ */
+@Category({ MiscTests.class, LargeTests.class })
+public class TestBulkLoadHFilesSplitRecovery {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestBulkLoadHFilesSplitRecovery.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestHRegionServerBulkLoad.class);
+
+  static HBaseTestingUtility util;
+  // used by secure subclass
+  static boolean useSecure = false;
+
+  final static int NUM_CFS = 10;
+  final static byte[] QUAL = Bytes.toBytes("qual");
+  final static int ROWCOUNT = 100;
+
+  private final static byte[][] families = new byte[NUM_CFS][];
+
+  @Rule
+  public TestName name = new TestName();
+
+  static {
+    for (int i = 0; i < NUM_CFS; i++) {
+      families[i] = Bytes.toBytes(family(i));
+    }
+  }
+
+  static byte[] rowkey(int i) {
+    return Bytes.toBytes(String.format("row_%08d", i));
+  }
+
+  static String family(int i) {
+    return String.format("family_%04d", i);
+  }
+
+  static byte[] value(int i) {
+    return Bytes.toBytes(String.format("%010d", i));
+  }
+
+  public static void buildHFiles(FileSystem fs, Path dir, int value) throws IOException {
+    byte[] val = value(value);
+    for (int i = 0; i < NUM_CFS; i++) {
+      Path testIn = new Path(dir, family(i));
+
+      TestHRegionServerBulkLoad.createHFile(fs, new Path(testIn, "hfile_" + i),
+        Bytes.toBytes(family(i)), QUAL, val, ROWCOUNT);
+    }
+  }
+
+  private TableDescriptor createTableDesc(TableName name, int cfs) {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name);
+    IntStream.range(0, cfs).mapToObj(i -> ColumnFamilyDescriptorBuilder.of(family(i)))
+      .forEachOrdered(builder::setColumnFamily);
+    return builder.build();
+  }
+
+  /**
+   * Creates a table with given table name and specified number of column families if the table does
+   * not already exist.
+   */
+  private void setupTable(final Connection connection, TableName table, int cfs)
+      throws IOException {
+    try {
+      LOG.info("Creating table " + table);
+      try (Admin admin = connection.getAdmin()) {
+        admin.createTable(createTableDesc(table, cfs));
+      }
+    } catch (TableExistsException tee) {
+      LOG.info("Table " + table + " already exists");
+    }
+  }
+
+  /**
+   * Creates a table with given table name,specified number of column families<br>
+   * and splitkeys if the table does not already exist.
+   * @param table
+   * @param cfs
+   * @param SPLIT_KEYS
+   */
+  private void setupTableWithSplitkeys(TableName table, int cfs, byte[][] SPLIT_KEYS)
+      throws IOException {
+    try {
+      LOG.info("Creating table " + table);
+      util.createTable(createTableDesc(table, cfs), SPLIT_KEYS);
+    } catch (TableExistsException tee) {
+      LOG.info("Table " + table + " already exists");
+    }
+  }
+
+  private Path buildBulkFiles(TableName table, int value) throws Exception {
+    Path dir = util.getDataTestDirOnTestFS(table.getNameAsString());
+    Path bulk1 = new Path(dir, table.getNameAsString() + value);
+    FileSystem fs = util.getTestFileSystem();
+    buildHFiles(fs, bulk1, value);
+    return bulk1;
+  }
+
+  /**
+   * Populate table with known values.
+   */
+  private void populateTable(final Connection connection, TableName table, int value)
+      throws Exception {
+    // create HFiles for different column families
+    Path dir = buildBulkFiles(table, value);
+    BulkLoadHFiles.create(util.getConfiguration()).bulkLoad(table, dir);
+  }
+
+  /**
+   * Split the known table in half. (this is hard coded for this test suite)
+   */
+  private void forceSplit(TableName table) {
+    try {
+      // need to call regions server to by synchronous but isn't visible.
+      HRegionServer hrs = util.getRSForFirstRegionInTable(table);
+
+      for (RegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
+        if (hri.getTable().equals(table)) {
+          util.getAdmin().splitRegionAsync(hri.getRegionName(), rowkey(ROWCOUNT / 2));
+          // ProtobufUtil.split(null, hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2));
+        }
+      }
+
+      // verify that split completed.
+      int regions;
+      do {
+        regions = 0;
+        for (RegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
+          if (hri.getTable().equals(table)) {
+            regions++;
+          }
+        }
+        if (regions != 2) {
+          LOG.info("Taking some time to complete split...");
+          Thread.sleep(250);
+        }
+      } while (regions != 2);
+    } catch (IOException e) {
+      e.printStackTrace();
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    util = new HBaseTestingUtility();
+    util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
+    util.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws Exception {
+    util.shutdownMiniCluster();
+  }
+
+  /**
+   * Checks that all columns have the expected value and that there is the expected number of rows.
+   * @throws IOException
+   */
+  void assertExpectedTable(TableName table, int count, int value) throws IOException {
+    TableDescriptor htd = util.getAdmin().getDescriptor(table);
+    assertNotNull(htd);
+    try (Table t = util.getConnection().getTable(table);
+        ResultScanner sr = t.getScanner(new Scan())) {
+      int i = 0;
+      for (Result r; (r = sr.next()) != null;) {
+        r.getNoVersionMap().values().stream().flatMap(m -> m.values().stream())
+          .forEach(v -> assertArrayEquals(value(value), v));
+        i++;
+      }
+      assertEquals(count, i);
+    } catch (IOException e) {
+      fail("Failed due to exception");
+    }
+  }
+
+  private static <T> CompletableFuture<T> failedFuture(Throwable error) {
+    CompletableFuture<T> future = new CompletableFuture<>();
+    future.completeExceptionally(error);
+    return future;
+  }
+
+  private static AsyncClusterConnection mockAndInjectError(AsyncClusterConnection conn) {
+    AsyncClusterConnection errConn = spy(conn);
+    doReturn(failedFuture(new IOException("injecting bulk load error"))).when(errConn)
+      .bulkLoad(any(), anyList(), any(), anyBoolean(), any(), any(), anyBoolean());
+    return errConn;
+  }
+
+  /**
+   * Test that shows that exception thrown from the RS side will result in an exception on the
+   * LIHFile client.
+   */
+  @Test(expected = IOException.class)
+  public void testBulkLoadPhaseFailure() throws Exception {
+    final TableName table = TableName.valueOf(name.getMethodName());
+    final AtomicInteger attemptedCalls = new AtomicInteger();
+    Configuration conf = new Configuration(util.getConfiguration());
+    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
+    BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf) {
+
+      @Override
+      protected void bulkLoadPhase(AsyncClusterConnection conn, TableName tableName,
+          Deque<LoadQueueItem> queue, Multimap<ByteBuffer, LoadQueueItem> regionGroups,
+          boolean copyFiles, Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
+        AsyncClusterConnection c =
+          attemptedCalls.incrementAndGet() == 1 ? mockAndInjectError(conn) : conn;
+        super.bulkLoadPhase(c, tableName, queue, regionGroups, copyFiles, item2RegionMap);
+      }
+    };
+    Path dir = buildBulkFiles(table, 1);
+    loader.bulkLoad(table, dir);
+  }
+
+  /**
+   * Test that shows that exception thrown from the RS side will result in the expected number of
+   * retries set by ${@link HConstants#HBASE_CLIENT_RETRIES_NUMBER} when
+   * ${@link BulkLoadHFiles#RETRY_ON_IO_EXCEPTION} is set
+   */
+  @Test
+  public void testRetryOnIOException() throws Exception {
+    TableName table = TableName.valueOf(name.getMethodName());
+    AtomicInteger calls = new AtomicInteger(0);
+    setupTable(util.getConnection(), table, 10);
+    Configuration conf = new Configuration(util.getConfiguration());
+    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
+    conf.setBoolean(BulkLoadHFiles.RETRY_ON_IO_EXCEPTION, true);
+    BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf) {
+
+      @Override
+      protected void bulkLoadPhase(AsyncClusterConnection conn, TableName tableName,
+          Deque<LoadQueueItem> queue, Multimap<ByteBuffer, LoadQueueItem> regionGroups,
+          boolean copyFiles, Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
+        if (calls.get() < conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+          HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER)) {
+          calls.incrementAndGet();
+          super.bulkLoadPhase(mockAndInjectError(conn), tableName, queue, regionGroups, copyFiles,
+            item2RegionMap);
+        } else {
+          super.bulkLoadPhase(conn, tableName, queue, regionGroups, copyFiles, item2RegionMap);
+        }
+      }
+    };
+    Path dir = buildBulkFiles(table, 1);
+    loader.bulkLoad(table, dir);
+    assertEquals(calls.get(), 2);
+  }
+
+  /**
+   * This test exercises the path where there is a split after initial validation but before the
+   * atomic bulk load call. We cannot use presplitting to test this path, so we actually inject a
+   * split just before the atomic region load.
+   */
+  @Test
+  public void testSplitWhileBulkLoadPhase() throws Exception {
+    final TableName table = TableName.valueOf(name.getMethodName());
+    setupTable(util.getConnection(), table, 10);
+    populateTable(util.getConnection(), table, 1);
+    assertExpectedTable(table, ROWCOUNT, 1);
+
+    // Now let's cause trouble. This will occur after checks and cause bulk
+    // files to fail when attempt to atomically import. This is recoverable.
+    final AtomicInteger attemptedCalls = new AtomicInteger();
+    BulkLoadHFilesTool loader = new BulkLoadHFilesTool(util.getConfiguration()) {
+
+      @Override
+      protected void bulkLoadPhase(AsyncClusterConnection conn, TableName tableName,
+          Deque<LoadQueueItem> queue, Multimap<ByteBuffer, LoadQueueItem> regionGroups,
+          boolean copyFiles, Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
+        int i = attemptedCalls.incrementAndGet();
+        if (i == 1) {
+          // On first attempt force a split.
+          forceSplit(table);
+        }
+        super.bulkLoadPhase(conn, tableName, queue, regionGroups, copyFiles, item2RegionMap);
+      }
+    };
+
+    // create HFiles for different column families
+    Path dir = buildBulkFiles(table, 2);
+    loader.bulkLoad(table, dir);
+
+    // check that data was loaded
+    // The three expected attempts are 1) failure because need to split, 2)
+    // load of split top 3) load of split bottom
+    assertEquals(3, attemptedCalls.get());
+    assertExpectedTable(table, ROWCOUNT, 2);
+  }
+
+  /**
+   * This test splits a table and attempts to bulk load. The bulk import files should be split
+   * before atomically importing.
+   */
+  @Test
+  public void testGroupOrSplitPresplit() throws Exception {
+    final TableName table = TableName.valueOf(name.getMethodName());
+    setupTable(util.getConnection(), table, 10);
+    populateTable(util.getConnection(), table, 1);
+    assertExpectedTable(util.getConnection(), table, ROWCOUNT, 1);
+    forceSplit(table);
+
+    final AtomicInteger countedLqis = new AtomicInteger();
+    BulkLoadHFilesTool loader = new BulkLoadHFilesTool(util.getConfiguration()) {
+
+      @Override
+      protected Pair<List<LoadQueueItem>, String> groupOrSplit(AsyncClusterConnection conn,
+          TableName tableName, Multimap<ByteBuffer, LoadQueueItem> regionGroups, LoadQueueItem item,
+          List<Pair<byte[], byte[]>> startEndKeys) throws IOException {
+        Pair<List<LoadQueueItem>, String> lqis =
+          super.groupOrSplit(conn, tableName, regionGroups, item, startEndKeys);
+        if (lqis != null && lqis.getFirst() != null) {
+          countedLqis.addAndGet(lqis.getFirst().size());
+        }
+        return lqis;
+      }
+    };
+
+    // create HFiles for different column families
+    Path dir = buildBulkFiles(table, 2);
+    loader.bulkLoad(table, dir);
+    assertExpectedTable(util.getConnection(), table, ROWCOUNT, 2);
+    assertEquals(20, countedLqis.get());
+  }
+
+  /**
+   * This test creates a table with many small regions. The bulk load files would be splitted
+   * multiple times before all of them can be loaded successfully.
+   */
+  @Test
+  public void testSplitTmpFileCleanUp() throws Exception {
+    final TableName table = TableName.valueOf(name.getMethodName());
+    byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("row_00000010"),
+      Bytes.toBytes("row_00000020"), Bytes.toBytes("row_00000030"), Bytes.toBytes("row_00000040"),
+      Bytes.toBytes("row_00000050") };
+    setupTableWithSplitkeys(table, 10, SPLIT_KEYS);
+
+    BulkLoadHFiles loader = BulkLoadHFiles.create(util.getConfiguration());
+
+    // create HFiles
+    Path dir = buildBulkFiles(table, 2);
+    loader.bulkLoad(table, dir);
+    // family path
+    Path tmpPath = new Path(dir, family(0));
+    // TMP_DIR under family path
+    tmpPath = new Path(tmpPath, BulkLoadHFilesTool.TMP_DIR);
+    FileSystem fs = dir.getFileSystem(util.getConfiguration());
+    // HFiles have been splitted, there is TMP_DIR
+    assertTrue(fs.exists(tmpPath));
+    // TMP_DIR should have been cleaned-up
+    assertNull(BulkLoadHFilesTool.TMP_DIR + " should be empty.", FSUtils.listStatus(fs, tmpPath));
+    assertExpectedTable(util.getConnection(), table, ROWCOUNT, 2);
+  }
+
+  /**
+   * This simulates an remote exception which should cause LIHF to exit with an exception.
+   */
+  @Test(expected = IOException.class)
+  public void testGroupOrSplitFailure() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    setupTable(util.getConnection(), tableName, 10);
+    BulkLoadHFilesTool loader = new BulkLoadHFilesTool(util.getConfiguration()) {
+
+      private int i = 0;
+
+      @Override
+      protected Pair<List<LoadQueueItem>, String> groupOrSplit(AsyncClusterConnection conn,
+          TableName tableName, Multimap<ByteBuffer, LoadQueueItem> regionGroups, LoadQueueItem item,
+          List<Pair<byte[], byte[]>> startEndKeys) throws IOException {
+        i++;
+
+        if (i == 5) {
+          throw new IOException("failure");
+        }
+        return super.groupOrSplit(conn, tableName, regionGroups, item, startEndKeys);
+      }
+    };
+
+    // create HFiles for different column families
+    Path dir = buildBulkFiles(tableName, 1);
+    loader.bulkLoad(tableName, dir);
+  }
+
+  /**
+   * Checks that all columns have the expected value and that there is the expected number of rows.
+   */
+  void assertExpectedTable(final Connection connection, TableName table, int count, int value)
+      throws IOException {
+    TableDescriptor htd = util.getAdmin().getDescriptor(table);
+    assertNotNull(htd);
+    try (Table t = connection.getTable(table); ResultScanner sr = t.getScanner(new Scan())) {
+      int i = 0;
+      for (Result r; (r = sr.next()) != null;) {
+        r.getNoVersionMap().values().stream().flatMap(m -> m.values().stream())
+          .forEach(v -> assertArrayEquals(value(value), v));
+        i++;
+      }
+      assertEquals(count, i);
+    } catch (IOException e) {
+      fail("Failed due to exception");
+    }
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java
deleted file mode 100644
index fcc1bb8..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java
+++ /dev/null
@@ -1,630 +0,0 @@
-/**
- * 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.tool;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.Deque;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.IntStream;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableExistsException;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClientServiceCallable;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-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.HConnectionTestingUtility;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.log.HBaseMarkers;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.Pair;
-import org.junit.AfterClass;
-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.mockito.Mockito;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
-
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
-
-/**
- * Test cases for the atomic load error handling of the bulk load functionality.
- */
-@Category({ MiscTests.class, LargeTests.class })
-public class TestLoadIncrementalHFilesSplitRecovery {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestLoadIncrementalHFilesSplitRecovery.class);
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestHRegionServerBulkLoad.class);
-
-  static HBaseTestingUtility util;
-  // used by secure subclass
-  static boolean useSecure = false;
-
-  final static int NUM_CFS = 10;
-  final static byte[] QUAL = Bytes.toBytes("qual");
-  final static int ROWCOUNT = 100;
-
-  private final static byte[][] families = new byte[NUM_CFS][];
-
-  @Rule
-  public TestName name = new TestName();
-
-  static {
-    for (int i = 0; i < NUM_CFS; i++) {
-      families[i] = Bytes.toBytes(family(i));
-    }
-  }
-
-  static byte[] rowkey(int i) {
-    return Bytes.toBytes(String.format("row_%08d", i));
-  }
-
-  static String family(int i) {
-    return String.format("family_%04d", i);
-  }
-
-  static byte[] value(int i) {
-    return Bytes.toBytes(String.format("%010d", i));
-  }
-
-  public static void buildHFiles(FileSystem fs, Path dir, int value) throws IOException {
-    byte[] val = value(value);
-    for (int i = 0; i < NUM_CFS; i++) {
-      Path testIn = new Path(dir, family(i));
-
-      TestHRegionServerBulkLoad.createHFile(fs, new Path(testIn, "hfile_" + i),
-        Bytes.toBytes(family(i)), QUAL, val, ROWCOUNT);
-    }
-  }
-
-  private TableDescriptor createTableDesc(TableName name, int cfs) {
-    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name);
-    IntStream.range(0, cfs).mapToObj(i -> ColumnFamilyDescriptorBuilder.of(family(i)))
-        .forEachOrdered(builder::setColumnFamily);
-    return builder.build();
-  }
-
-  /**
-   * Creates a table with given table name and specified number of column families if the table does
-   * not already exist.
-   */
-  private void setupTable(final Connection connection, TableName table, int cfs)
-      throws IOException {
-    try {
-      LOG.info("Creating table " + table);
-      try (Admin admin = connection.getAdmin()) {
-        admin.createTable(createTableDesc(table, cfs));
-      }
-    } catch (TableExistsException tee) {
-      LOG.info("Table " + table + " already exists");
-    }
-  }
-
-  /**
-   * Creates a table with given table name,specified number of column families<br>
-   * and splitkeys if the table does not already exist.
-   * @param table
-   * @param cfs
-   * @param SPLIT_KEYS
-   */
-  private void setupTableWithSplitkeys(TableName table, int cfs, byte[][] SPLIT_KEYS)
-      throws IOException {
-    try {
-      LOG.info("Creating table " + table);
-      util.createTable(createTableDesc(table, cfs), SPLIT_KEYS);
-    } catch (TableExistsException tee) {
-      LOG.info("Table " + table + " already exists");
-    }
-  }
-
-  private Path buildBulkFiles(TableName table, int value) throws Exception {
-    Path dir = util.getDataTestDirOnTestFS(table.getNameAsString());
-    Path bulk1 = new Path(dir, table.getNameAsString() + value);
-    FileSystem fs = util.getTestFileSystem();
-    buildHFiles(fs, bulk1, value);
-    return bulk1;
-  }
-
-  /**
-   * Populate table with known values.
-   */
-  private void populateTable(final Connection connection, TableName table, int value)
-      throws Exception {
-    // create HFiles for different column families
-    LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration());
-    Path bulk1 = buildBulkFiles(table, value);
-    try (Table t = connection.getTable(table);
-        RegionLocator locator = connection.getRegionLocator(table);
-        Admin admin = connection.getAdmin()) {
-      lih.doBulkLoad(bulk1, admin, t, locator);
-    }
-  }
-
-  /**
-   * Split the known table in half. (this is hard coded for this test suite)
-   */
-  private void forceSplit(TableName table) {
-    try {
-      // need to call regions server to by synchronous but isn't visible.
-      HRegionServer hrs = util.getRSForFirstRegionInTable(table);
-
-      for (RegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
-        if (hri.getTable().equals(table)) {
-          util.getAdmin().splitRegionAsync(hri.getRegionName(), rowkey(ROWCOUNT / 2));
-          // ProtobufUtil.split(null, hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2));
-        }
-      }
-
-      // verify that split completed.
-      int regions;
-      do {
-        regions = 0;
-        for (RegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
-          if (hri.getTable().equals(table)) {
-            regions++;
-          }
-        }
-        if (regions != 2) {
-          LOG.info("Taking some time to complete split...");
-          Thread.sleep(250);
-        }
-      } while (regions != 2);
-    } catch (IOException e) {
-      e.printStackTrace();
-    } catch (InterruptedException e) {
-      e.printStackTrace();
-    }
-  }
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    util = new HBaseTestingUtility();
-    util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
-    util.startMiniCluster(1);
-  }
-
-  @AfterClass
-  public static void teardownCluster() throws Exception {
-    util.shutdownMiniCluster();
-  }
-
-  /**
-   * Checks that all columns have the expected value and that there is the expected number of rows.
-   * @throws IOException
-   */
-  void assertExpectedTable(TableName table, int count, int value) throws IOException {
-    TableDescriptor htd = util.getAdmin().getDescriptor(table);
-    assertNotNull(htd);
-    try (Table t = util.getConnection().getTable(table);
-        ResultScanner sr = t.getScanner(new Scan())) {
-      int i = 0;
-      for (Result r; (r = sr.next()) != null;) {
-        r.getNoVersionMap().values().stream().flatMap(m -> m.values().stream())
-            .forEach(v -> assertArrayEquals(value(value), v));
-        i++;
-      }
-      assertEquals(count, i);
-    } catch (IOException e) {
-      fail("Failed due to exception");
-    }
-  }
-
-  /**
-   * Test that shows that exception thrown from the RS side will result in an exception on the
-   * LIHFile client.
-   */
-  @Test(expected = IOException.class)
-  public void testBulkLoadPhaseFailure() throws Exception {
-    final TableName table = TableName.valueOf(name.getMethodName());
-    final AtomicInteger attmptedCalls = new AtomicInteger();
-    final AtomicInteger failedCalls = new AtomicInteger();
-    util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
-    try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
-      setupTable(connection, table, 10);
-      LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration()) {
-        @Override
-        protected List<LoadQueueItem> tryAtomicRegionLoad(
-            ClientServiceCallable<byte[]> serviceCallable, TableName tableName, final byte[] first,
-            Collection<LoadQueueItem> lqis) throws IOException {
-          int i = attmptedCalls.incrementAndGet();
-          if (i == 1) {
-            Connection errConn;
-            try {
-              errConn = getMockedConnection(util.getConfiguration());
-              serviceCallable = this.buildClientServiceCallable(errConn, table, first, lqis, true);
-            } catch (Exception e) {
-              LOG.error(HBaseMarkers.FATAL, "mocking cruft, should never happen", e);
-              throw new RuntimeException("mocking cruft, should never happen");
-            }
-            failedCalls.incrementAndGet();
-            return super.tryAtomicRegionLoad(serviceCallable, tableName, first, lqis);
-          }
-
-          return super.tryAtomicRegionLoad(serviceCallable, tableName, first, lqis);
-        }
-      };
-      try {
-        // create HFiles for different column families
-        Path dir = buildBulkFiles(table, 1);
-        try (Table t = connection.getTable(table);
-            RegionLocator locator = connection.getRegionLocator(table);
-            Admin admin = connection.getAdmin()) {
-          lih.doBulkLoad(dir, admin, t, locator);
-        }
-      } finally {
-        util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
-          HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
-      }
-      fail("doBulkLoad should have thrown an exception");
-    }
-  }
-
-  /**
-   * Test that shows that exception thrown from the RS side will result in the expected number of
-   * retries set by ${@link HConstants#HBASE_CLIENT_RETRIES_NUMBER} when
-   * ${@link LoadIncrementalHFiles#RETRY_ON_IO_EXCEPTION} is set
-   */
-  @Test
-  public void testRetryOnIOException() throws Exception {
-    final TableName table = TableName.valueOf(name.getMethodName());
-    final AtomicInteger calls = new AtomicInteger(0);
-    final Connection conn = ConnectionFactory.createConnection(util.getConfiguration());
-    util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
-    util.getConfiguration().setBoolean(LoadIncrementalHFiles.RETRY_ON_IO_EXCEPTION, true);
-    final LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration()) {
-      @Override
-      protected List<LoadQueueItem> tryAtomicRegionLoad(
-          ClientServiceCallable<byte[]> serverCallable, TableName tableName, final byte[] first,
-          Collection<LoadQueueItem> lqis) throws IOException {
-        if (calls.get() < util.getConfiguration().getInt(
-          HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER)) {
-          ClientServiceCallable<byte[]> newServerCallable = new ClientServiceCallable<byte[]>(conn,
-              tableName, first, new RpcControllerFactory(util.getConfiguration()).newController(),
-              HConstants.PRIORITY_UNSET) {
-            @Override
-            public byte[] rpcCall() throws Exception {
-              throw new IOException("Error calling something on RegionServer");
-            }
-          };
-          calls.getAndIncrement();
-          return super.tryAtomicRegionLoad(newServerCallable, tableName, first, lqis);
-        } else {
-          return super.tryAtomicRegionLoad(serverCallable, tableName, first, lqis);
-        }
-      }
-    };
-    setupTable(conn, table, 10);
-    Path dir = buildBulkFiles(table, 1);
-    lih.doBulkLoad(dir, conn.getAdmin(), conn.getTable(table), conn.getRegionLocator(table));
-    assertEquals(calls.get(), 2);
-    util.getConfiguration().setBoolean(LoadIncrementalHFiles.RETRY_ON_IO_EXCEPTION, false);
-  }
-
-  private ClusterConnection getMockedConnection(final Configuration conf)
-      throws IOException, org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
-    ServerName sn = ServerName.valueOf("example.org", 1234, 0);
-    RegionInfo hri = RegionInfoBuilder.FIRST_META_REGIONINFO;
-    ClientProtos.ClientService.BlockingInterface client =
-      Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
-    Mockito
-      .when(
-        client.bulkLoadHFile((RpcController) Mockito.any(), (BulkLoadHFileRequest) Mockito.any()))
-      .thenThrow(new ServiceException(new IOException("injecting bulk load error")));
-    return HConnectionTestingUtility.getMockedConnectionAndDecorate(conf, null, client, sn, hri);
-  }
-
-  /**
-   * This test exercises the path where there is a split after initial validation but before the
-   * atomic bulk load call. We cannot use presplitting to test this path, so we actually inject a
-   * split just before the atomic region load.
-   */
-  @Test
-  public void testSplitWhileBulkLoadPhase() throws Exception {
-    final TableName table = TableName.valueOf(name.getMethodName());
-    try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
-      setupTable(connection, table, 10);
-      populateTable(connection, table, 1);
-      assertExpectedTable(table, ROWCOUNT, 1);
-
-      // Now let's cause trouble. This will occur after checks and cause bulk
-      // files to fail when attempt to atomically import. This is recoverable.
-      final AtomicInteger attemptedCalls = new AtomicInteger();
-      LoadIncrementalHFiles lih2 = new LoadIncrementalHFiles(util.getConfiguration()) {
-        @Override
-        protected void bulkLoadPhase(final Table htable, final Connection conn,
-            ExecutorService pool, Deque<LoadQueueItem> queue,
-            final Multimap<ByteBuffer, LoadQueueItem> regionGroups, boolean copyFile,
-            Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
-          int i = attemptedCalls.incrementAndGet();
-          if (i == 1) {
-            // On first attempt force a split.
-            forceSplit(table);
-          }
-          super.bulkLoadPhase(htable, conn, pool, queue, regionGroups, copyFile, item2RegionMap);
-        }
-      };
-
-      // create HFiles for different column families
-      try (Table t = connection.getTable(table);
-          RegionLocator locator = connection.getRegionLocator(table);
-          Admin admin = connection.getAdmin()) {
-        Path bulk = buildBulkFiles(table, 2);
-        lih2.doBulkLoad(bulk, admin, t, locator);
-      }
-
-      // check that data was loaded
-      // The three expected attempts are 1) failure because need to split, 2)
-      // load of split top 3) load of split bottom
-      assertEquals(3, attemptedCalls.get());
-      assertExpectedTable(table, ROWCOUNT, 2);
-    }
-  }
-
-  /**
-   * This test splits a table and attempts to bulk load. The bulk import files should be split
-   * before atomically importing.
-   */
-  @Test
-  public void testGroupOrSplitPresplit() throws Exception {
-    final TableName table = TableName.valueOf(name.getMethodName());
-    try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
-      setupTable(connection, table, 10);
-      populateTable(connection, table, 1);
-      assertExpectedTable(connection, table, ROWCOUNT, 1);
-      forceSplit(table);
-
-      final AtomicInteger countedLqis = new AtomicInteger();
-      LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration()) {
-        @Override
-        protected Pair<List<LoadQueueItem>, String> groupOrSplit(
-            Multimap<ByteBuffer, LoadQueueItem> regionGroups, final LoadQueueItem item,
-            final Table htable, final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
-          Pair<List<LoadQueueItem>, String> lqis =
-              super.groupOrSplit(regionGroups, item, htable, startEndKeys);
-          if (lqis != null && lqis.getFirst() != null) {
-            countedLqis.addAndGet(lqis.getFirst().size());
-          }
-          return lqis;
-        }
-      };
-
-      // create HFiles for different column families
-      Path bulk = buildBulkFiles(table, 2);
-      try (Table t = connection.getTable(table);
-          RegionLocator locator = connection.getRegionLocator(table);
-          Admin admin = connection.getAdmin()) {
-        lih.doBulkLoad(bulk, admin, t, locator);
-      }
-      assertExpectedTable(connection, table, ROWCOUNT, 2);
-      assertEquals(20, countedLqis.get());
-    }
-  }
-
-  /**
-   * This test creates a table with many small regions. The bulk load files would be splitted
-   * multiple times before all of them can be loaded successfully.
-   */
-  @Test
-  public void testSplitTmpFileCleanUp() throws Exception {
-    final TableName table = TableName.valueOf(name.getMethodName());
-    byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("row_00000010"),
-        Bytes.toBytes("row_00000020"), Bytes.toBytes("row_00000030"), Bytes.toBytes("row_00000040"),
-        Bytes.toBytes("row_00000050") };
-    try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
-      setupTableWithSplitkeys(table, 10, SPLIT_KEYS);
-
-      LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration());
-
-      // create HFiles
-      Path bulk = buildBulkFiles(table, 2);
-      try (Table t = connection.getTable(table);
-          RegionLocator locator = connection.getRegionLocator(table);
-          Admin admin = connection.getAdmin()) {
-        lih.doBulkLoad(bulk, admin, t, locator);
-      }
-      // family path
-      Path tmpPath = new Path(bulk, family(0));
-      // TMP_DIR under family path
-      tmpPath = new Path(tmpPath, LoadIncrementalHFiles.TMP_DIR);
-      FileSystem fs = bulk.getFileSystem(util.getConfiguration());
-      // HFiles have been splitted, there is TMP_DIR
-      assertTrue(fs.exists(tmpPath));
-      // TMP_DIR should have been cleaned-up
-      assertNull(LoadIncrementalHFiles.TMP_DIR + " should be empty.",
-        FSUtils.listStatus(fs, tmpPath));
-      assertExpectedTable(connection, table, ROWCOUNT, 2);
-    }
-  }
-
-  /**
-   * This simulates an remote exception which should cause LIHF to exit with an exception.
-   */
-  @Test(expected = IOException.class)
-  public void testGroupOrSplitFailure() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
-      setupTable(connection, tableName, 10);
-
-      LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration()) {
-        int i = 0;
-
-        @Override
-        protected Pair<List<LoadQueueItem>, String> groupOrSplit(
-            Multimap<ByteBuffer, LoadQueueItem> regionGroups, final LoadQueueItem item,
-            final Table table, final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
-          i++;
-
-          if (i == 5) {
-            throw new IOException("failure");
-          }
-          return super.groupOrSplit(regionGroups, item, table, startEndKeys);
-        }
-      };
-
-      // create HFiles for different column families
-      Path dir = buildBulkFiles(tableName, 1);
-      try (Table t = connection.getTable(tableName);
-          RegionLocator locator = connection.getRegionLocator(tableName);
-          Admin admin = connection.getAdmin()) {
-        lih.doBulkLoad(dir, admin, t, locator);
-      }
-    }
-
-    fail("doBulkLoad should have thrown an exception");
-  }
-
-  @Test
-  public void testGroupOrSplitWhenRegionHoleExistsInMeta() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("row_00000100") };
-    // Share connection. We were failing to find the table with our new reverse scan because it
-    // looks for first region, not any region -- that is how it works now. The below removes first
-    // region in test. Was reliant on the Connection caching having first region.
-    Connection connection = ConnectionFactory.createConnection(util.getConfiguration());
-    Table table = connection.getTable(tableName);
-
-    setupTableWithSplitkeys(tableName, 10, SPLIT_KEYS);
-    Path dir = buildBulkFiles(tableName, 2);
-
-    final AtomicInteger countedLqis = new AtomicInteger();
-    LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration()) {
-
-      @Override
-      protected Pair<List<LoadQueueItem>, String> groupOrSplit(
-          Multimap<ByteBuffer, LoadQueueItem> regionGroups, final LoadQueueItem item,
-          final Table htable, final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
-        Pair<List<LoadQueueItem>, String> lqis =
-            super.groupOrSplit(regionGroups, item, htable, startEndKeys);
-        if (lqis != null && lqis.getFirst() != null) {
-          countedLqis.addAndGet(lqis.getFirst().size());
-        }
-        return lqis;
-      }
-    };
-
-    // do bulkload when there is no region hole in hbase:meta.
-    try (Table t = connection.getTable(tableName);
-        RegionLocator locator = connection.getRegionLocator(tableName);
-        Admin admin = connection.getAdmin()) {
-      loader.doBulkLoad(dir, admin, t, locator);
-    } catch (Exception e) {
-      LOG.error("exeception=", e);
-    }
-    // check if all the data are loaded into the table.
-    this.assertExpectedTable(tableName, ROWCOUNT, 2);
-
-    dir = buildBulkFiles(tableName, 3);
-
-    // Mess it up by leaving a hole in the hbase:meta
-    List<RegionInfo> regionInfos = MetaTableAccessor.getTableRegions(connection, tableName);
-    for (RegionInfo regionInfo : regionInfos) {
-      if (Bytes.equals(regionInfo.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
-        MetaTableAccessor.deleteRegion(connection, regionInfo);
-        break;
-      }
-    }
-
-    try (Table t = connection.getTable(tableName);
-        RegionLocator locator = connection.getRegionLocator(tableName);
-        Admin admin = connection.getAdmin()) {
-      loader.doBulkLoad(dir, admin, t, locator);
-    } catch (Exception e) {
-      LOG.error("exception=", e);
-      assertTrue("IOException expected", e instanceof IOException);
-    }
-
-    table.close();
-
-    // Make sure at least the one region that still exists can be found.
-    regionInfos = MetaTableAccessor.getTableRegions(connection, tableName);
-    assertTrue(regionInfos.size() >= 1);
-
-    this.assertExpectedTable(connection, tableName, ROWCOUNT, 2);
-    connection.close();
-  }
-
-  /**
-   * Checks that all columns have the expected value and that there is the expected number of rows.
-   * @throws IOException
-   */
-  void assertExpectedTable(final Connection connection, TableName table, int count, int value)
-      throws IOException {
-    TableDescriptor htd = util.getAdmin().getDescriptor(table);
-    assertNotNull(htd);
-    try (Table t = connection.getTable(table); ResultScanner sr = t.getScanner(new Scan())) {
-      int i = 0;
-      for (Result r; (r = sr.next()) != null;) {
-        r.getNoVersionMap().values().stream().flatMap(m -> m.values().stream())
-            .forEach(v -> assertArrayEquals(value(value), v));
-        i++;
-      }
-      assertEquals(count, i);
-    } catch (IOException e) {
-      fail("Failed due to exception");
-    }
-  }
-}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureBulkLoadHFiles.java
similarity index 88%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFiles.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureBulkLoadHFiles.java
index 4e10f01..2ef808a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFiles.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureBulkLoadHFiles.java
@@ -31,20 +31,20 @@ import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 
 /**
- * Reruns TestLoadIncrementalHFiles using LoadIncrementalHFiles in secure mode. This suite is unable
+ * Reruns TestBulkLoadHFiles using BulkLoadHFiles in secure mode. This suite is unable
  * to verify the security handoff/turnover as miniCluster is running as system user thus has root
  * privileges and delegation tokens don't seem to work on miniDFS.
- * <p>
+ * <p/>
  * Thus SecureBulkload can only be completely verified by running integration tests against a secure
  * cluster. This suite is still invaluable as it verifies the other mechanisms that need to be
  * supported as part of a LoadIncrementalFiles call.
  */
 @Category({ MiscTests.class, LargeTests.class })
-public class TestSecureLoadIncrementalHFiles extends TestLoadIncrementalHFiles {
+public class TestSecureBulkLoadHFiles extends TestBulkLoadHFiles {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestSecureLoadIncrementalHFiles.class);
+    HBaseClassTestRule.forClass(TestSecureBulkLoadHFiles.class);
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -53,7 +53,7 @@ public class TestSecureLoadIncrementalHFiles extends TestLoadIncrementalHFiles {
       HadoopSecurityEnabledUserProviderForTesting.class);
     // setup configuration
     SecureTestUtil.enableSecurity(util.getConfiguration());
-    util.getConfiguration().setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY,
+    util.getConfiguration().setInt(BulkLoadHFiles.MAX_FILES_PER_REGION_PER_FAMILY,
       MAX_FILES_PER_REGION_PER_FAMILY);
     // change default behavior so that tag values are returned with normal rpcs
     util.getConfiguration().set(HConstants.RPC_CODEC_CONF_KEY,
@@ -66,5 +66,4 @@ public class TestSecureLoadIncrementalHFiles extends TestLoadIncrementalHFiles {
 
     setupNamespace();
   }
-
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureBulkLoadHFilesSplitRecovery.java
similarity index 90%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFilesSplitRecovery.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureBulkLoadHFilesSplitRecovery.java
index 7fe79a9..65ba8ff 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFilesSplitRecovery.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureBulkLoadHFilesSplitRecovery.java
@@ -31,21 +31,20 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 /**
- * Reruns TestSecureLoadIncrementalHFilesSplitRecovery using LoadIncrementalHFiles in secure mode.
+ * Reruns TestBulkLoadHFilesSplitRecovery using BulkLoadHFiles in secure mode.
  * This suite is unable to verify the security handoff/turnove as miniCluster is running as system
  * user thus has root privileges and delegation tokens don't seem to work on miniDFS.
- * <p>
+ * <p/>
  * Thus SecureBulkload can only be completely verified by running integration tests against a secure
  * cluster. This suite is still invaluable as it verifies the other mechanisms that need to be
  * supported as part of a LoadIncrementalFiles call.
  */
 @Category({ MiscTests.class, LargeTests.class })
-public class TestSecureLoadIncrementalHFilesSplitRecovery
-    extends TestLoadIncrementalHFilesSplitRecovery {
+public class TestSecureBulkLoadHFilesSplitRecovery extends TestBulkLoadHFilesSplitRecovery {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestSecureLoadIncrementalHFilesSplitRecovery.class);
+    HBaseClassTestRule.forClass(TestSecureBulkLoadHFilesSplitRecovery.class);
 
   // This "overrides" the parent static method
   // make sure they are in sync


[hbase] 08/11: HBASE-21719 Rewrite RegionPlacementMaintainer to use AsyncClusterConnection

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

zhangduo pushed a commit to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 64ea281891ce78d07bfdb58a7580700182fcc9ed
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Tue Jan 15 11:43:41 2019 +0800

    HBASE-21719 Rewrite RegionPlacementMaintainer to use AsyncClusterConnection
    
    Signed-off-by: Michael Stack <st...@apache.org>
---
 .../hbase/master/RegionPlacementMaintainer.java    | 225 +++++++++++----------
 1 file changed, 113 insertions(+), 112 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
index faf5e4a..fda0a9c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
@@ -1,5 +1,4 @@
 /**
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -16,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.text.DecimalFormat;
 import java.util.ArrayList;
@@ -39,29 +38,30 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
+import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
 import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.MunkresAssignment;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.GnuParser;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException;
 
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
 
@@ -71,7 +71,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavor
  */
 @InterfaceAudience.Private
 // TODO: Remove? Unused. Partially implemented only.
-public class RegionPlacementMaintainer {
+public class RegionPlacementMaintainer implements Closeable {
   private static final Logger LOG = LoggerFactory.getLogger(RegionPlacementMaintainer.class
       .getName());
   //The cost of a placement that should never be assigned.
@@ -96,9 +96,9 @@ public class RegionPlacementMaintainer {
   private final boolean enforceMinAssignmentMove;
   private RackManager rackManager;
   private Set<TableName> targetTableSet;
-  private final Connection connection;
+  private AsyncClusterConnection connection;
 
-  public RegionPlacementMaintainer(Configuration conf) {
+  public RegionPlacementMaintainer(Configuration conf) throws IOException {
     this(conf, true, true);
   }
 
@@ -109,11 +109,6 @@ public class RegionPlacementMaintainer {
     this.enforceMinAssignmentMove = enforceMinAssignmentMove;
     this.targetTableSet = new HashSet<>();
     this.rackManager = new RackManager(conf);
-    try {
-      this.connection = ConnectionFactory.createConnection(this.conf);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
   }
 
   private static void printHelp(Options opt) {
@@ -124,6 +119,14 @@ public class RegionPlacementMaintainer {
         " [-fs hdfs://a.b.c.d:9000] [-hbase_root /HBASE]", opt);
   }
 
+  private AsyncClusterConnection getConnection() throws IOException {
+    if (connection == null) {
+      connection =
+        ClusterConnectionFactory.createAsyncClusterConnection(this.conf, null, User.getCurrent());
+    }
+    return connection;
+  }
+
   public void setTargetTableName(String[] tableNames) {
     if (tableNames != null) {
       for (String table : tableNames)
@@ -133,10 +136,8 @@ public class RegionPlacementMaintainer {
 
   /**
    * @return the new RegionAssignmentSnapshot
-   * @throws IOException
    */
-  public SnapshotOfRegionAssignmentFromMeta getRegionAssignmentSnapshot()
-  throws IOException {
+  public SnapshotOfRegionAssignmentFromMeta getRegionAssignmentSnapshot() throws IOException {
     SnapshotOfRegionAssignmentFromMeta currentAssignmentShapshot =
       new SnapshotOfRegionAssignmentFromMeta(ConnectionFactory.createConnection(conf));
     currentAssignmentShapshot.initialize();
@@ -145,9 +146,6 @@ public class RegionPlacementMaintainer {
 
   /**
    * Verify the region placement is consistent with the assignment plan
-   * @param isDetailMode
-   * @return reports
-   * @throws IOException
    */
   public List<AssignmentVerificationReport> verifyRegionPlacement(boolean isDetailMode)
       throws IOException {
@@ -206,10 +204,9 @@ public class RegionPlacementMaintainer {
 
     // Get the all the region servers
     List<ServerName> servers = new ArrayList<>();
-    try (Admin admin = this.connection.getAdmin()) {
-      servers.addAll(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+    servers.addAll(
+      FutureUtils.get(getConnection().getAdmin().getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)))
         .getLiveServerMetrics().keySet());
-    }
 
     LOG.info("Start to generate assignment plan for " + numRegions +
         " regions from table " + tableName + " with " +
@@ -492,6 +489,11 @@ public class RegionPlacementMaintainer {
     return plan;
   }
 
+  @Override
+  public void close() throws IOException {
+    Closeables.close(connection, true);
+  }
+
   /**
    * Some algorithms for solving the assignment problem may traverse workers or
    * jobs in linear order which may result in skewing the assignments of the
@@ -690,19 +692,17 @@ public class RegionPlacementMaintainer {
         }
         if (singleServerPlan != null) {
           // Update the current region server with its updated favored nodes
-          BlockingInterface currentRegionServer =
-            ((ClusterConnection)this.connection).getAdmin(entry.getKey());
+          AsyncRegionServerAdmin rsAdmin = getConnection().getRegionServerAdmin(entry.getKey());
           UpdateFavoredNodesRequest request =
-              RequestConverter.buildUpdateFavoredNodesRequest(regionUpdateInfos);
-
+            RequestConverter.buildUpdateFavoredNodesRequest(regionUpdateInfos);
           UpdateFavoredNodesResponse updateFavoredNodesResponse =
-              currentRegionServer.updateFavoredNodes(null, request);
+            FutureUtils.get(rsAdmin.updateFavoredNodes(request));
           LOG.info("Region server " +
-              ProtobufUtil.getServerInfo(null, currentRegionServer).getServerName() +
-              " has updated " + updateFavoredNodesResponse.getResponse() + " / " +
-              singleServerPlan.getAssignmentMap().size() +
-              " regions with the assignment plan");
-          succeededNum ++;
+            FutureUtils.get(rsAdmin.getServerInfo(RequestConverter.buildGetServerInfoRequest()))
+              .getServerInfo() +
+            " has updated " + updateFavoredNodesResponse.getResponse() + " / " +
+            singleServerPlan.getAssignmentMap().size() + " regions with the assignment plan");
+          succeededNum++;
         }
       } catch (Exception e) {
         failedUpdateMap.put(entry.getKey(), e);
@@ -719,7 +719,7 @@ public class RegionPlacementMaintainer {
           " region servers with its corresponding favored nodes");
       for (Map.Entry<ServerName, Exception> entry :
         failedUpdateMap.entrySet() ) {
-        LOG.error("Failed to update " + entry.getKey().getHostAndPort() +
+        LOG.error("Failed to update " + entry.getKey().getAddress() +
             " because of " + entry.getValue().getMessage());
       }
     }
@@ -1019,93 +1019,94 @@ public class RegionPlacementMaintainer {
       }
 
       // Create the region placement obj
-      RegionPlacementMaintainer rp = new RegionPlacementMaintainer(conf, enforceLocality,
-          enforceMinAssignmentMove);
+      try (RegionPlacementMaintainer rp =
+        new RegionPlacementMaintainer(conf, enforceLocality, enforceMinAssignmentMove)) {
 
-      if (cmd.hasOption("d") || cmd.hasOption("verification-details")) {
-        verificationDetails = true;
-      }
-
-      if (cmd.hasOption("tables")) {
-        String tableNameListStr = cmd.getOptionValue("tables");
-        String[] tableNames = StringUtils.split(tableNameListStr, ",");
-        rp.setTargetTableName(tableNames);
-      }
+        if (cmd.hasOption("d") || cmd.hasOption("verification-details")) {
+          verificationDetails = true;
+        }
 
-      if (cmd.hasOption("munkres")) {
-        USE_MUNKRES_FOR_PLACING_SECONDARY_AND_TERTIARY = true;
-      }
+        if (cmd.hasOption("tables")) {
+          String tableNameListStr = cmd.getOptionValue("tables");
+          String[] tableNames = StringUtils.split(tableNameListStr, ",");
+          rp.setTargetTableName(tableNames);
+        }
 
-      // Read all the modes
-      if (cmd.hasOption("v") || cmd.hasOption("verify")) {
-        // Verify the region placement.
-        rp.verifyRegionPlacement(verificationDetails);
-      } else if (cmd.hasOption("n") || cmd.hasOption("dry-run")) {
-        // Generate the assignment plan only without updating the hbase:meta and RS
-        FavoredNodesPlan plan = rp.getNewAssignmentPlan();
-        printAssignmentPlan(plan);
-      } else if (cmd.hasOption("w") || cmd.hasOption("write")) {
-        // Generate the new assignment plan
-        FavoredNodesPlan plan = rp.getNewAssignmentPlan();
-        // Print the new assignment plan
-        printAssignmentPlan(plan);
-        // Write the new assignment plan to META
-        rp.updateAssignmentPlanToMeta(plan);
-      } else if (cmd.hasOption("u") || cmd.hasOption("update")) {
-        // Generate the new assignment plan
-        FavoredNodesPlan plan = rp.getNewAssignmentPlan();
-        // Print the new assignment plan
-        printAssignmentPlan(plan);
-        // Update the assignment to hbase:meta and Region Servers
-        rp.updateAssignmentPlan(plan);
-      } else if (cmd.hasOption("diff")) {
-        FavoredNodesPlan newPlan = rp.getNewAssignmentPlan();
-        Map<String, Map<String, Float>> locality = FSUtils
-            .getRegionDegreeLocalityMappingFromFS(conf);
-        Map<TableName, Integer> movesPerTable = rp.getRegionsMovement(newPlan);
-        rp.checkDifferencesWithOldPlan(movesPerTable, locality, newPlan);
-        System.out.println("Do you want to update the assignment plan? [y/n]");
-        Scanner s = new Scanner(System.in);
-        String input = s.nextLine().trim();
-        if (input.equals("y")) {
-          System.out.println("Updating assignment plan...");
-          rp.updateAssignmentPlan(newPlan);
+        if (cmd.hasOption("munkres")) {
+          USE_MUNKRES_FOR_PLACING_SECONDARY_AND_TERTIARY = true;
         }
-        s.close();
-      } else if (cmd.hasOption("ld")) {
-        Map<String, Map<String, Float>> locality = FSUtils
-            .getRegionDegreeLocalityMappingFromFS(conf);
-        rp.printLocalityAndDispersionForCurrentPlan(locality);
-      } else if (cmd.hasOption("p") || cmd.hasOption("print")) {
-        FavoredNodesPlan plan = rp.getRegionAssignmentSnapshot().getExistingAssignmentPlan();
-        printAssignmentPlan(plan);
-      } else if (cmd.hasOption("overwrite")) {
-        if (!cmd.hasOption("f") || !cmd.hasOption("r")) {
-          throw new IllegalArgumentException("Please specify: " +
+
+        // Read all the modes
+        if (cmd.hasOption("v") || cmd.hasOption("verify")) {
+          // Verify the region placement.
+          rp.verifyRegionPlacement(verificationDetails);
+        } else if (cmd.hasOption("n") || cmd.hasOption("dry-run")) {
+          // Generate the assignment plan only without updating the hbase:meta and RS
+          FavoredNodesPlan plan = rp.getNewAssignmentPlan();
+          printAssignmentPlan(plan);
+        } else if (cmd.hasOption("w") || cmd.hasOption("write")) {
+          // Generate the new assignment plan
+          FavoredNodesPlan plan = rp.getNewAssignmentPlan();
+          // Print the new assignment plan
+          printAssignmentPlan(plan);
+          // Write the new assignment plan to META
+          rp.updateAssignmentPlanToMeta(plan);
+        } else if (cmd.hasOption("u") || cmd.hasOption("update")) {
+          // Generate the new assignment plan
+          FavoredNodesPlan plan = rp.getNewAssignmentPlan();
+          // Print the new assignment plan
+          printAssignmentPlan(plan);
+          // Update the assignment to hbase:meta and Region Servers
+          rp.updateAssignmentPlan(plan);
+        } else if (cmd.hasOption("diff")) {
+          FavoredNodesPlan newPlan = rp.getNewAssignmentPlan();
+          Map<String, Map<String, Float>> locality =
+            FSUtils.getRegionDegreeLocalityMappingFromFS(conf);
+          Map<TableName, Integer> movesPerTable = rp.getRegionsMovement(newPlan);
+          rp.checkDifferencesWithOldPlan(movesPerTable, locality, newPlan);
+          System.out.println("Do you want to update the assignment plan? [y/n]");
+          Scanner s = new Scanner(System.in);
+          String input = s.nextLine().trim();
+          if (input.equals("y")) {
+            System.out.println("Updating assignment plan...");
+            rp.updateAssignmentPlan(newPlan);
+          }
+          s.close();
+        } else if (cmd.hasOption("ld")) {
+          Map<String, Map<String, Float>> locality =
+            FSUtils.getRegionDegreeLocalityMappingFromFS(conf);
+          rp.printLocalityAndDispersionForCurrentPlan(locality);
+        } else if (cmd.hasOption("p") || cmd.hasOption("print")) {
+          FavoredNodesPlan plan = rp.getRegionAssignmentSnapshot().getExistingAssignmentPlan();
+          printAssignmentPlan(plan);
+        } else if (cmd.hasOption("overwrite")) {
+          if (!cmd.hasOption("f") || !cmd.hasOption("r")) {
+            throw new IllegalArgumentException("Please specify: " +
               " -update -r regionName -f server1:port,server2:port,server3:port");
-        }
+          }
 
-        String regionName = cmd.getOptionValue("r");
-        String favoredNodesStr = cmd.getOptionValue("f");
-        LOG.info("Going to update the region " + regionName + " with the new favored nodes " +
+          String regionName = cmd.getOptionValue("r");
+          String favoredNodesStr = cmd.getOptionValue("f");
+          LOG.info("Going to update the region " + regionName + " with the new favored nodes " +
             favoredNodesStr);
-        List<ServerName> favoredNodes = null;
-        RegionInfo regionInfo =
+          List<ServerName> favoredNodes = null;
+          RegionInfo regionInfo =
             rp.getRegionAssignmentSnapshot().getRegionNameToRegionInfoMap().get(regionName);
-        if (regionInfo == null) {
-          LOG.error("Cannot find the region " + regionName + " from the META");
-        } else {
-          try {
-            favoredNodes = getFavoredNodeList(favoredNodesStr);
-          } catch (IllegalArgumentException e) {
-            LOG.error("Cannot parse the invalid favored nodes because " + e);
+          if (regionInfo == null) {
+            LOG.error("Cannot find the region " + regionName + " from the META");
+          } else {
+            try {
+              favoredNodes = getFavoredNodeList(favoredNodesStr);
+            } catch (IllegalArgumentException e) {
+              LOG.error("Cannot parse the invalid favored nodes because " + e);
+            }
+            FavoredNodesPlan newPlan = new FavoredNodesPlan();
+            newPlan.updateFavoredNodesMap(regionInfo, favoredNodes);
+            rp.updateAssignmentPlan(newPlan);
           }
-          FavoredNodesPlan newPlan = new FavoredNodesPlan();
-          newPlan.updateFavoredNodesMap(regionInfo, favoredNodes);
-          rp.updateAssignmentPlan(newPlan);
+        } else {
+          printHelp(opt);
         }
-      } else {
-        printHelp(opt);
       }
     } catch (ParseException e) {
       printHelp(opt);


[hbase] 06/11: HBASE-21671 Rewrite RegionReplicaReplicationEndpoint to use AsyncClusterConnection

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

zhangduo pushed a commit to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit f83e7af019b8a0f70854d629f01518f98eaba4c1
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Fri Jan 11 16:22:24 2019 +0800

    HBASE-21671 Rewrite RegionReplicaReplicationEndpoint to use AsyncClusterConnection
---
 .../hadoop/hbase/client/AsyncConnectionImpl.java   |  25 +-
 .../hbase/client/AsyncClusterConnection.java       |  17 +
 .../hbase/client/AsyncClusterConnectionImpl.java   |  80 +++
 .../AsyncRegionReplicaReplayRetryingCaller.java    | 147 ++++
 .../hbase/client/AsyncRegionServerAdmin.java       |   5 +-
 .../hbase/client/ClusterConnectionFactory.java     |   2 +-
 .../hbase/protobuf/ReplicationProtbufUtil.java     |  31 +-
 .../handler/RegionReplicaFlushHandler.java         |   3 +-
 .../hbase/replication/ReplicationEndpoint.java     |  35 +-
 .../RegionReplicaReplicationEndpoint.java          | 782 +++++++--------------
 .../regionserver/ReplicationSource.java            |   2 +-
 .../TestRegionReplicaReplicationEndpoint.java      |  56 +-
 ...stRegionReplicaReplicationEndpointNoMaster.java |  99 ++-
 13 files changed, 627 insertions(+), 657 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 8386ffa..7c91e49 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
@@ -55,7 +55,6 @@ import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
 
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
@@ -65,7 +64,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterServ
  * The implementation of AsyncConnection.
  */
 @InterfaceAudience.Private
-class AsyncConnectionImpl implements AsyncClusterConnection {
+class AsyncConnectionImpl implements AsyncConnection {
 
   private static final Logger LOG = LoggerFactory.getLogger(AsyncConnectionImpl.class);
 
@@ -85,7 +84,7 @@ class AsyncConnectionImpl implements AsyncClusterConnection {
 
   private final int rpcTimeout;
 
-  private final RpcClient rpcClient;
+  protected final RpcClient rpcClient;
 
   final RpcControllerFactory rpcControllerFactory;
 
@@ -173,16 +172,10 @@ class AsyncConnectionImpl implements AsyncClusterConnection {
   }
 
   // ditto
-  @Override
-  public NonceGenerator getNonceGenerator() {
+  NonceGenerator getNonceGenerator() {
     return nonceGenerator;
   }
 
-  @Override
-  public RpcClient getRpcClient() {
-    return rpcClient;
-  }
-
   private ClientService.Interface createRegionServerStub(ServerName serverName) throws IOException {
     return ClientService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));
   }
@@ -378,16 +371,4 @@ class AsyncConnectionImpl implements AsyncClusterConnection {
   public void clearRegionLocationCache() {
     locator.clearCache();
   }
-
-  @Override
-  public AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName) {
-    return new AsyncRegionServerAdmin(serverName, this);
-  }
-
-  @Override
-  public CompletableFuture<FlushRegionResponse> flush(byte[] regionName,
-      boolean writeFlushWALMarker) {
-    RawAsyncHBaseAdmin admin = (RawAsyncHBaseAdmin) getAdmin();
-    return admin.flushRegionInternal(regionName, writeFlushWALMarker);
-  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
similarity index 72%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
rename to hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
index f1f64ca..0ad77ba 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
@@ -17,9 +17,13 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.util.List;
 import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
@@ -49,4 +53,17 @@ public interface AsyncClusterConnection extends AsyncConnection {
    * Flush a region and get the response.
    */
   CompletableFuture<FlushRegionResponse> flush(byte[] regionName, boolean writeFlushWALMarker);
+
+  /**
+   * Replicate wal edits for replica regions. The return value is the edits we skipped, as the
+   * original return value is useless.
+   */
+  CompletableFuture<Long> replay(TableName tableName, byte[] encodedRegionName, byte[] row,
+      List<Entry> entries, int replicaId, int numRetries, long operationTimeoutNs);
+
+  /**
+   * Return all the replicas for a region. Used for regiong replica replication.
+   */
+  CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
+      boolean reload);
 }
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
new file mode 100644
index 0000000..d61f01f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java
@@ -0,0 +1,80 @@
+/**
+ * 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 java.net.SocketAddress;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
+
+/**
+ * The implementation of AsyncClusterConnection.
+ */
+@InterfaceAudience.Private
+class AsyncClusterConnectionImpl extends AsyncConnectionImpl implements AsyncClusterConnection {
+
+  public AsyncClusterConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId,
+      SocketAddress localAddress, User user) {
+    super(conf, registry, clusterId, localAddress, user);
+  }
+
+  @Override
+  public NonceGenerator getNonceGenerator() {
+    return super.getNonceGenerator();
+  }
+
+  @Override
+  public RpcClient getRpcClient() {
+    return rpcClient;
+  }
+
+  @Override
+  public AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName) {
+    return new AsyncRegionServerAdmin(serverName, this);
+  }
+
+  @Override
+  public CompletableFuture<FlushRegionResponse> flush(byte[] regionName,
+      boolean writeFlushWALMarker) {
+    RawAsyncHBaseAdmin admin = (RawAsyncHBaseAdmin) getAdmin();
+    return admin.flushRegionInternal(regionName, writeFlushWALMarker);
+  }
+
+  @Override
+  public CompletableFuture<Long> replay(TableName tableName, byte[] encodedRegionName, byte[] row,
+      List<Entry> entries, int replicaId, int retries, long operationTimeoutNs) {
+    return new AsyncRegionReplicaReplayRetryingCaller(RETRY_TIMER, this,
+      ConnectionUtils.retries2Attempts(retries), operationTimeoutNs, tableName, encodedRegionName,
+      row, entries, replicaId).call();
+  }
+
+  @Override
+  public CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
+      boolean reload) {
+    return getLocator().getRegionLocations(tableName, row, RegionLocateType.CURRENT, reload, -1L);
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionReplicaReplayRetryingCaller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionReplicaReplayRetryingCaller.java
new file mode 100644
index 0000000..3853292
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionReplicaReplayRetryingCaller.java
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
+
+/**
+ * For replaying edits for region replica.
+ * <p/>
+ * The mainly difference here is that, every time after locating, we will check whether the region
+ * name is equal, if not, we will give up, as this usually means the region has been split or
+ * merged, and the new region(s) should already have all the data of the parent region(s).
+ * <p/>
+ * Notice that, the return value is the edits we skipped, as the original response message is not
+ * used at upper layer.
+ */
+@InterfaceAudience.Private
+public class AsyncRegionReplicaReplayRetryingCaller extends AsyncRpcRetryingCaller<Long> {
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(AsyncRegionReplicaReplayRetryingCaller.class);
+
+  private final TableName tableName;
+
+  private final byte[] encodedRegionName;
+
+  private final byte[] row;
+
+  private final Entry[] entries;
+
+  private final int replicaId;
+
+  public AsyncRegionReplicaReplayRetryingCaller(HashedWheelTimer retryTimer,
+      AsyncClusterConnectionImpl conn, int maxAttempts, long operationTimeoutNs,
+      TableName tableName, byte[] encodedRegionName, byte[] row, List<Entry> entries,
+      int replicaId) {
+    super(retryTimer, conn, ConnectionUtils.getPriority(tableName), conn.connConf.getPauseNs(),
+      maxAttempts, operationTimeoutNs, conn.connConf.getWriteRpcTimeoutNs(),
+      conn.connConf.getStartLogErrorsCnt());
+    this.tableName = tableName;
+    this.encodedRegionName = encodedRegionName;
+    this.row = row;
+    this.entries = entries.toArray(new Entry[0]);
+    this.replicaId = replicaId;
+  }
+
+  private void call(HRegionLocation loc) {
+    if (!Bytes.equals(encodedRegionName, loc.getRegion().getEncodedNameAsBytes())) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(
+          "Skipping {} entries in table {} because located region {} is different than" +
+            " the original region {} from WALEdit",
+          entries.length, tableName, loc.getRegion().getEncodedName(),
+          Bytes.toStringBinary(encodedRegionName));
+        for (Entry entry : entries) {
+          LOG.trace("Skipping : " + entry);
+        }
+      }
+      future.complete(Long.valueOf(entries.length));
+      return;
+    }
+
+    AdminService.Interface stub;
+    try {
+      stub = conn.getAdminStub(loc.getServerName());
+    } catch (IOException e) {
+      onError(e,
+        () -> "Get async admin stub to " + loc.getServerName() + " for '" +
+          Bytes.toStringBinary(row) + "' in " + loc.getRegion().getEncodedName() + " of " +
+          tableName + " failed",
+        err -> conn.getLocator().updateCachedLocationOnError(loc, err));
+      return;
+    }
+    Pair<ReplicateWALEntryRequest, CellScanner> p = ReplicationProtbufUtil
+      .buildReplicateWALEntryRequest(entries, encodedRegionName, null, null, null);
+    resetCallTimeout();
+    controller.setCellScanner(p.getSecond());
+    stub.replay(controller, p.getFirst(), r -> {
+      if (controller.failed()) {
+        onError(controller.getFailed(),
+          () -> "Call to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row) + "' in " +
+            loc.getRegion().getEncodedName() + " of " + tableName + " failed",
+          err -> conn.getLocator().updateCachedLocationOnError(loc, err));
+      } else {
+        future.complete(0L);
+      }
+    });
+
+  }
+
+  @Override
+  protected void doCall() {
+    long locateTimeoutNs;
+    if (operationTimeoutNs > 0) {
+      locateTimeoutNs = remainingTimeNs();
+      if (locateTimeoutNs <= 0) {
+        completeExceptionally();
+        return;
+      }
+    } else {
+      locateTimeoutNs = -1L;
+    }
+    addListener(conn.getLocator().getRegionLocation(tableName, row, replicaId,
+      RegionLocateType.CURRENT, locateTimeoutNs), (loc, error) -> {
+        if (error != null) {
+          onError(error,
+            () -> "Locate '" + Bytes.toStringBinary(row) + "' in " + tableName + " failed", err -> {
+            });
+          return;
+        }
+        call(loc);
+      });
+  }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
similarity index 99%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
rename to hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
index b9141a9..d491890 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
@@ -164,8 +164,9 @@ public class AsyncRegionServerAdmin {
       cellScanner);
   }
 
-  public CompletableFuture<ReplicateWALEntryResponse> replay(ReplicateWALEntryRequest request) {
-    return call((stub, controller, done) -> stub.replay(controller, request, done));
+  public CompletableFuture<ReplicateWALEntryResponse> replay(ReplicateWALEntryRequest request,
+      CellScanner cellScanner) {
+    return call((stub, controller, done) -> stub.replay(controller, request, done), cellScanner);
   }
 
   public CompletableFuture<RollWALWriterResponse> rollWALWriter(RollWALWriterRequest request) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
similarity index 95%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
rename to hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
index 79484db..2670420 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
@@ -46,6 +46,6 @@ public final class ClusterConnectionFactory {
       SocketAddress localAddress, User user) throws IOException {
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf);
     String clusterId = FutureUtils.get(registry.getClusterId());
-    return new AsyncConnectionImpl(conf, registry, clusterId, localAddress, user);
+    return new AsyncClusterConnectionImpl(conf, registry, clusterId, localAddress, user);
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
index 9f41a76..c39c86c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
@@ -37,7 +37,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 
 @InterfaceAudience.Private
@@ -55,20 +56,18 @@ public class ReplicationProtbufUtil {
   public static void replicateWALEntry(AsyncRegionServerAdmin admin, Entry[] entries,
       String replicationClusterId, Path sourceBaseNamespaceDir, Path sourceHFileArchiveDir)
       throws IOException {
-    Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p = buildReplicateWALEntryRequest(
-      entries, null, replicationClusterId, sourceBaseNamespaceDir, sourceHFileArchiveDir);
+    Pair<ReplicateWALEntryRequest, CellScanner> p = buildReplicateWALEntryRequest(entries, null,
+      replicationClusterId, sourceBaseNamespaceDir, sourceHFileArchiveDir);
     FutureUtils.get(admin.replicateWALEntry(p.getFirst(), p.getSecond()));
   }
 
   /**
    * Create a new ReplicateWALEntryRequest from a list of WAL entries
-   *
    * @param entries the WAL entries to be replicated
-   * @return a pair of ReplicateWALEntryRequest and a CellScanner over all the WALEdit values
-   * found.
+   * @return a pair of ReplicateWALEntryRequest and a CellScanner over all the WALEdit values found.
    */
-  public static Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner>
-      buildReplicateWALEntryRequest(final Entry[] entries) throws IOException {
+  public static Pair<ReplicateWALEntryRequest, CellScanner> buildReplicateWALEntryRequest(
+      final Entry[] entries) {
     return buildReplicateWALEntryRequest(entries, null, null, null, null);
   }
 
@@ -82,16 +81,14 @@ public class ReplicationProtbufUtil {
    * @param sourceHFileArchiveDir Path to the source cluster hfile archive directory
    * @return a pair of ReplicateWALEntryRequest and a CellScanner over all the WALEdit values found.
    */
-  public static Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner>
-      buildReplicateWALEntryRequest(final Entry[] entries, byte[] encodedRegionName,
-          String replicationClusterId, Path sourceBaseNamespaceDir, Path sourceHFileArchiveDir)
-          throws IOException {
+  public static Pair<ReplicateWALEntryRequest, CellScanner> buildReplicateWALEntryRequest(
+      final Entry[] entries, byte[] encodedRegionName, String replicationClusterId,
+      Path sourceBaseNamespaceDir, Path sourceHFileArchiveDir) {
     // Accumulate all the Cells seen in here.
     List<List<? extends Cell>> allCells = new ArrayList<>(entries.length);
     int size = 0;
-    AdminProtos.WALEntry.Builder entryBuilder = AdminProtos.WALEntry.newBuilder();
-    AdminProtos.ReplicateWALEntryRequest.Builder builder =
-      AdminProtos.ReplicateWALEntryRequest.newBuilder();
+    WALEntry.Builder entryBuilder = WALEntry.newBuilder();
+    ReplicateWALEntryRequest.Builder builder = ReplicateWALEntryRequest.newBuilder();
 
     for (Entry entry: entries) {
       entryBuilder.clear();
@@ -99,8 +96,8 @@ public class ReplicationProtbufUtil {
       try {
         keyBuilder = entry.getKey().getBuilder(WALCellCodec.getNoneCompressor());
       } catch (IOException e) {
-        throw new IOException(
-            "There should not throw exception since NoneCompressor do not throw any exceptions", e);
+        throw new AssertionError(
+          "There should not throw exception since NoneCompressor do not throw any exceptions", e);
       }
       if(encodedRegionName != null){
         keyBuilder.setEncodedRegionName(
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RegionReplicaFlushHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RegionReplicaFlushHandler.java
index 0729203..cc798cc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RegionReplicaFlushHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RegionReplicaFlushHandler.java
@@ -185,7 +185,6 @@ public class RegionReplicaFlushHandler extends EventHandler {
             "Was not able to trigger a flush from primary region due to old server version? " +
               "Continuing to open the secondary region replica: " +
               region.getRegionInfo().getRegionNameAsString());
-          region.setReadsEnabled(true);
           break;
         }
       }
@@ -195,6 +194,6 @@ public class RegionReplicaFlushHandler extends EventHandler {
         throw new InterruptedIOException(e.getMessage());
       }
     }
+    region.setReadsEnabled(true);
   }
-
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java
index f4c37b1..ca73663 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java
@@ -29,6 +29,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
@@ -53,6 +54,7 @@ public interface ReplicationEndpoint extends ReplicationPeerConfigListener {
 
   @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
   class Context {
+    private final Server server;
     private final Configuration localConf;
     private final Configuration conf;
     private final FileSystem fs;
@@ -64,16 +66,11 @@ public interface ReplicationEndpoint extends ReplicationPeerConfigListener {
     private final Abortable abortable;
 
     @InterfaceAudience.Private
-    public Context(
-        final Configuration localConf,
-        final Configuration conf,
-        final FileSystem fs,
-        final String peerId,
-        final UUID clusterId,
-        final ReplicationPeer replicationPeer,
-        final MetricsSource metrics,
-        final TableDescriptors tableDescriptors,
-        final Abortable abortable) {
+    public Context(final Server server, final Configuration localConf, final Configuration conf,
+        final FileSystem fs, final String peerId, final UUID clusterId,
+        final ReplicationPeer replicationPeer, final MetricsSource metrics,
+        final TableDescriptors tableDescriptors, final Abortable abortable) {
+      this.server = server;
       this.localConf = localConf;
       this.conf = conf;
       this.fs = fs;
@@ -84,34 +81,50 @@ public interface ReplicationEndpoint extends ReplicationPeerConfigListener {
       this.tableDescriptors = tableDescriptors;
       this.abortable = abortable;
     }
+
+    public Server getServer() {
+      return server;
+    }
+
     public Configuration getConfiguration() {
       return conf;
     }
+
     public Configuration getLocalConfiguration() {
       return localConf;
     }
+
     public FileSystem getFilesystem() {
       return fs;
     }
+
     public UUID getClusterId() {
       return clusterId;
     }
+
     public String getPeerId() {
       return peerId;
     }
+
     public ReplicationPeerConfig getPeerConfig() {
       return replicationPeer.getPeerConfig();
     }
+
     public ReplicationPeer getReplicationPeer() {
       return replicationPeer;
     }
+
     public MetricsSource getMetrics() {
       return metrics;
     }
+
     public TableDescriptors getTableDescriptors() {
       return tableDescriptors;
     }
-    public Abortable getAbortable() { return abortable; }
+
+    public Abortable getAbortable() {
+      return abortable;
+    }
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
index f7721e0..65cf9a8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
@@ -19,67 +19,47 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
-import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.Callable;
+import java.util.Optional;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.RegionAdminServiceCallable;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.client.RetryingCallable;
-import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
+import org.apache.hadoop.hbase.util.AtomicUtils;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.hadoop.hbase.util.RetryCounterFactory;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers;
-import org.apache.hadoop.hbase.wal.WALSplitter.OutputSink;
-import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController;
-import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer;
-import org.apache.hadoop.hbase.wal.WALSplitter.SinkWriter;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
 import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
-
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
+import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
 
 /**
- * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint} endpoint
- * which receives the WAL edits from the WAL, and sends the edits to replicas
- * of regions.
+ * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint} endpoint which receives the WAL
+ * edits from the WAL, and sends the edits to replicas of regions.
  */
 @InterfaceAudience.Private
 public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
@@ -87,32 +67,55 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
   private static final Logger LOG = LoggerFactory.getLogger(RegionReplicaReplicationEndpoint.class);
 
   // Can be configured differently than hbase.client.retries.number
-  private static String CLIENT_RETRIES_NUMBER
-    = "hbase.region.replica.replication.client.retries.number";
+  private static String CLIENT_RETRIES_NUMBER =
+    "hbase.region.replica.replication.client.retries.number";
 
   private Configuration conf;
-  private ClusterConnection connection;
+  private AsyncClusterConnection connection;
   private TableDescriptors tableDescriptors;
 
-  // Reuse WALSplitter constructs as a WAL pipe
-  private PipelineController controller;
-  private RegionReplicaOutputSink outputSink;
-  private EntryBuffers entryBuffers;
+  private int numRetries;
+
+  private long operationTimeoutNs;
 
-  // Number of writer threads
-  private int numWriterThreads;
+  private LoadingCache<TableName, Optional<TableDescriptor>> tableDescriptorCache;
 
-  private int operationTimeout;
+  private Cache<TableName, TableName> disabledTableCache;
 
-  private ExecutorService pool;
+  private final RetryCounterFactory retryCounterFactory =
+    new RetryCounterFactory(Integer.MAX_VALUE, 1000, 60000);
 
   @Override
   public void init(Context context) throws IOException {
     super.init(context);
-
-    this.conf = HBaseConfiguration.create(context.getConfiguration());
+    this.conf = context.getConfiguration();
     this.tableDescriptors = context.getTableDescriptors();
-
+    int memstoreReplicationEnabledCacheExpiryMs = conf
+      .getInt("hbase.region.replica.replication.cache.memstoreReplicationEnabled.expiryMs", 5000);
+    // A cache for the table "memstore replication enabled" flag.
+    // It has a default expiry of 5 sec. This means that if the table is altered
+    // with a different flag value, we might miss to replicate for that amount of
+    // time. But this cache avoid the slow lookup and parsing of the TableDescriptor.
+    tableDescriptorCache = CacheBuilder.newBuilder()
+      .expireAfterWrite(memstoreReplicationEnabledCacheExpiryMs, TimeUnit.MILLISECONDS)
+      .initialCapacity(10).maximumSize(1000)
+      .build(new CacheLoader<TableName, Optional<TableDescriptor>>() {
+
+        @Override
+        public Optional<TableDescriptor> load(TableName tableName) throws Exception {
+          // check if the table requires memstore replication
+          // some unit-test drop the table, so we should do a bypass check and always replicate.
+          return Optional.ofNullable(tableDescriptors.get(tableName));
+        }
+      });
+    int nonExistentTableCacheExpiryMs =
+      conf.getInt("hbase.region.replica.replication.cache.disabledAndDroppedTables.expiryMs", 5000);
+    // A cache for non existing tables that have a default expiry of 5 sec. This means that if the
+    // table is created again with the same name, we might miss to replicate for that amount of
+    // time. But this cache prevents overloading meta requests for every edit from a deleted file.
+    disabledTableCache = CacheBuilder.newBuilder()
+      .expireAfterWrite(nonExistentTableCacheExpiryMs, TimeUnit.MILLISECONDS).initialCapacity(10)
+      .maximumSize(1000).build();
     // HRS multiplies client retries by 10 globally for meta operations, but we do not want this.
     // We are resetting it here because we want default number of retries (35) rather than 10 times
     // that which makes very long retries for disabled tables etc.
@@ -123,516 +126,261 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
         HConstants.DEFAULT_HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER);
       defaultNumRetries = defaultNumRetries / mult; // reset if HRS has multiplied this already
     }
-
-    conf.setInt(HConstants.HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER, 1);
-    int numRetries = conf.getInt(CLIENT_RETRIES_NUMBER, defaultNumRetries);
-    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, numRetries);
-
-    this.numWriterThreads = this.conf.getInt(
-      "hbase.region.replica.replication.writer.threads", 3);
-    controller = new PipelineController();
-    entryBuffers = new EntryBuffers(controller,
-        this.conf.getLong("hbase.region.replica.replication.buffersize", 128 * 1024 * 1024));
-
+    this.numRetries = conf.getInt(CLIENT_RETRIES_NUMBER, defaultNumRetries);
     // use the regular RPC timeout for replica replication RPC's
-    this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
-      HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
-  }
-
-  @Override
-  protected void doStart() {
-    try {
-      connection = (ClusterConnection) ConnectionFactory.createConnection(this.conf);
-      this.pool = getDefaultThreadPool(conf);
-      outputSink = new RegionReplicaOutputSink(controller, tableDescriptors, entryBuffers,
-        connection, pool, numWriterThreads, operationTimeout);
-      outputSink.startWriterThreads();
-      super.doStart();
-    } catch (IOException ex) {
-      LOG.warn("Received exception while creating connection :" + ex);
-      notifyFailed(ex);
-    }
-  }
-
-  @Override
-  protected void doStop() {
-    if (outputSink != null) {
-      try {
-        outputSink.finishWritingAndClose();
-      } catch (IOException ex) {
-        LOG.warn("Got exception while trying to close OutputSink", ex);
-      }
-    }
-    if (this.pool != null) {
-      this.pool.shutdownNow();
-      try {
-        // wait for 10 sec
-        boolean shutdown = this.pool.awaitTermination(10000, TimeUnit.MILLISECONDS);
-        if (!shutdown) {
-          LOG.warn("Failed to shutdown the thread pool after 10 seconds");
-        }
-      } catch (InterruptedException e) {
-        LOG.warn("Got interrupted while waiting for the thread pool to shut down" + e);
-      }
-    }
-    if (connection != null) {
-      try {
-        connection.close();
-      } catch (IOException ex) {
-        LOG.warn("Got exception closing connection :" + ex);
-      }
-    }
-    super.doStop();
+    this.operationTimeoutNs =
+      TimeUnit.MILLISECONDS.toNanos(conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
+        HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT));
+    this.connection = context.getServer().getAsyncClusterConnection();
   }
 
   /**
-   * Returns a Thread pool for the RPC's to region replicas. Similar to
-   * Connection's thread pool.
+   * returns true if the specified entry must be replicated. We should always replicate meta
+   * operations (e.g. flush) and use the user HTD flag to decide whether or not replicate the
+   * memstore.
    */
-  private ExecutorService getDefaultThreadPool(Configuration conf) {
-    int maxThreads = conf.getInt("hbase.region.replica.replication.threads.max", 256);
-    if (maxThreads == 0) {
-      maxThreads = Runtime.getRuntime().availableProcessors() * 8;
+  private boolean requiresReplication(Optional<TableDescriptor> tableDesc, Entry entry) {
+    // empty edit does not need to be replicated
+    if (entry.getEdit().isEmpty() || !tableDesc.isPresent()) {
+      return false;
     }
-    long keepAliveTime = conf.getLong("hbase.region.replica.replication.threads.keepalivetime", 60);
-    LinkedBlockingQueue<Runnable> workQueue =
-        new LinkedBlockingQueue<>(maxThreads *
-            conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
-              HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
-    ThreadPoolExecutor tpe = new ThreadPoolExecutor(
-      maxThreads,
-      maxThreads,
-      keepAliveTime,
-      TimeUnit.SECONDS,
-      workQueue,
-      Threads.newDaemonThreadFactory(this.getClass().getSimpleName() + "-rpc-shared-"));
-    tpe.allowCoreThreadTimeOut(true);
-    return tpe;
+    // meta edits (e.g. flush) must be always replicated
+    return entry.getEdit().isMetaEdit() || tableDesc.get().hasRegionMemStoreReplication();
   }
 
-  @Override
-  public boolean replicate(ReplicateContext replicateContext) {
-    /* A note on batching in RegionReplicaReplicationEndpoint (RRRE):
-     *
-     * RRRE relies on batching from two different mechanisms. The first is the batching from
-     * ReplicationSource since RRRE is a ReplicationEndpoint driven by RS. RS reads from a single
-     * WAL file filling up a buffer of heap size "replication.source.size.capacity"(64MB) or at most
-     * "replication.source.nb.capacity" entries or until it sees the end of file (in live tailing).
-     * Then RS passes all the buffered edits in this replicate() call context. RRRE puts the edits
-     * to the WALSplitter.EntryBuffers which is a blocking buffer space of up to
-     * "hbase.region.replica.replication.buffersize" (128MB) in size. This buffer splits the edits
-     * based on regions.
-     *
-     * There are "hbase.region.replica.replication.writer.threads"(default 3) writer threads which
-     * pick largest per-region buffer and send it to the SinkWriter (see RegionReplicaOutputSink).
-     * The SinkWriter in this case will send the wal edits to all secondary region replicas in
-     * parallel via a retrying rpc call. EntryBuffers guarantees that while a buffer is
-     * being written to the sink, another buffer for the same region will not be made available to
-     * writers ensuring regions edits are not replayed out of order.
-     *
-     * The replicate() call won't return until all the buffers are sent and ack'd by the sinks so
-     * that the replication can assume all edits are persisted. We may be able to do a better
-     * pipelining between the replication thread and output sinks later if it becomes a bottleneck.
-     */
-
-    while (this.isRunning()) {
-      try {
-        for (Entry entry: replicateContext.getEntries()) {
-          entryBuffers.appendEntry(entry);
+  private void getRegionLocations(CompletableFuture<RegionLocations> future,
+      TableDescriptor tableDesc, byte[] encodedRegionName, byte[] row, boolean reload) {
+    FutureUtils.addListener(connection.getRegionLocations(tableDesc.getTableName(), row, reload),
+      (r, e) -> {
+        if (e != null) {
+          future.completeExceptionally(e);
+          return;
         }
-        outputSink.flush(); // make sure everything is flushed
-        ctx.getMetrics().incrLogEditsFiltered(
-          outputSink.getSkippedEditsCounter().getAndSet(0));
-        return true;
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        return false;
-      } catch (IOException e) {
-        LOG.warn("Received IOException while trying to replicate"
-            + StringUtils.stringifyException(e));
-      }
-    }
-
-    return false;
-  }
-
-  @Override
-  public boolean canReplicateToSameCluster() {
-    return true;
-  }
-
-  @Override
-  protected WALEntryFilter getScopeWALEntryFilter() {
-    // we do not care about scope. We replicate everything.
-    return null;
+        // if we are not loading from cache, just return
+        if (reload) {
+          future.complete(r);
+          return;
+        }
+        // check if the number of region replicas is correct, and also the primary region name
+        // matches
+        if (r.size() == tableDesc.getRegionReplication() && Bytes.equals(
+          r.getDefaultRegionLocation().getRegion().getEncodedNameAsBytes(), encodedRegionName)) {
+          future.complete(r);
+        } else {
+          // reload again as the information in cache maybe stale
+          getRegionLocations(future, tableDesc, encodedRegionName, row, true);
+        }
+      });
   }
 
-  static class RegionReplicaOutputSink extends OutputSink {
-    private final RegionReplicaSinkWriter sinkWriter;
-    private final TableDescriptors tableDescriptors;
-    private final Cache<TableName, Boolean> memstoreReplicationEnabled;
-
-    public RegionReplicaOutputSink(PipelineController controller, TableDescriptors tableDescriptors,
-        EntryBuffers entryBuffers, ClusterConnection connection, ExecutorService pool,
-        int numWriters, int operationTimeout) {
-      super(controller, entryBuffers, numWriters);
-      this.sinkWriter =
-          new RegionReplicaSinkWriter(this, connection, pool, operationTimeout, tableDescriptors);
-      this.tableDescriptors = tableDescriptors;
-
-      // A cache for the table "memstore replication enabled" flag.
-      // It has a default expiry of 5 sec. This means that if the table is altered
-      // with a different flag value, we might miss to replicate for that amount of
-      // time. But this cache avoid the slow lookup and parsing of the TableDescriptor.
-      int memstoreReplicationEnabledCacheExpiryMs = connection.getConfiguration()
-        .getInt("hbase.region.replica.replication.cache.memstoreReplicationEnabled.expiryMs", 5000);
-      this.memstoreReplicationEnabled = CacheBuilder.newBuilder()
-        .expireAfterWrite(memstoreReplicationEnabledCacheExpiryMs, TimeUnit.MILLISECONDS)
-        .initialCapacity(10)
-        .maximumSize(1000)
-        .build();
+  private void replicate(CompletableFuture<Long> future, RegionLocations locs,
+      TableDescriptor tableDesc, byte[] encodedRegionName, byte[] row, List<Entry> entries) {
+    if (locs.size() == 1) {
+      // Could this happen?
+      future.complete(Long.valueOf(entries.size()));
+      return;
     }
-
-    @Override
-    public void append(RegionEntryBuffer buffer) throws IOException {
-      List<Entry> entries = buffer.getEntryBuffer();
-
-      if (entries.isEmpty() || entries.get(0).getEdit().getCells().isEmpty()) {
-        return;
-      }
-
-      // meta edits (e.g. flush) are always replicated.
-      // data edits (e.g. put) are replicated if the table requires them.
-      if (!requiresReplication(buffer.getTableName(), entries)) {
-        return;
+    if (!Bytes.equals(locs.getDefaultRegionLocation().getRegion().getEncodedNameAsBytes(),
+      encodedRegionName)) {
+      // the region name is not equal, this usually means the region has been split or merged, so
+      // give up replicating as the new region(s) should already have all the data of the parent
+      // region(s).
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(
+          "Skipping {} entries in table {} because located region {} is different than" +
+            " the original region {} from WALEdit",
+          tableDesc.getTableName(), locs.getDefaultRegionLocation().getRegion().getEncodedName(),
+          Bytes.toStringBinary(encodedRegionName));
       }
-
-      sinkWriter.append(buffer.getTableName(), buffer.getEncodedRegionName(),
-        CellUtil.cloneRow(entries.get(0).getEdit().getCells().get(0)), entries);
-    }
-
-    @Override
-    public boolean flush() throws IOException {
-      // nothing much to do for now. Wait for the Writer threads to finish up
-      // append()'ing the data.
-      entryBuffers.waitUntilDrained();
-      return super.flush();
-    }
-
-    @Override
-    public boolean keepRegionEvent(Entry entry) {
-      return true;
-    }
-
-    @Override
-    public List<Path> finishWritingAndClose() throws IOException {
-      finishWriting(true);
-      return null;
-    }
-
-    @Override
-    public Map<byte[], Long> getOutputCounts() {
-      return null; // only used in tests
-    }
-
-    @Override
-    public int getNumberOfRecoveredRegions() {
-      return 0;
-    }
-
-    AtomicLong getSkippedEditsCounter() {
-      return skippedEdits;
+      future.complete(Long.valueOf(entries.size()));
+      return;
     }
-
-    /**
-     * returns true if the specified entry must be replicated.
-     * We should always replicate meta operations (e.g. flush)
-     * and use the user HTD flag to decide whether or not replicate the memstore.
-     */
-    private boolean requiresReplication(final TableName tableName, final List<Entry> entries)
-        throws IOException {
-      // unit-tests may not the TableDescriptors, bypass the check and always replicate
-      if (tableDescriptors == null) return true;
-
-      Boolean requiresReplication = memstoreReplicationEnabled.getIfPresent(tableName);
-      if (requiresReplication == null) {
-        // check if the table requires memstore replication
-        // some unit-test drop the table, so we should do a bypass check and always replicate.
-        TableDescriptor htd = tableDescriptors.get(tableName);
-        requiresReplication = htd == null || htd.hasRegionMemStoreReplication();
-        memstoreReplicationEnabled.put(tableName, requiresReplication);
-      }
-
-      // if memstore replication is not required, check the entries.
-      // meta edits (e.g. flush) must be always replicated.
-      if (!requiresReplication) {
-        int skipEdits = 0;
-        java.util.Iterator<Entry> it = entries.iterator();
-        while (it.hasNext()) {
-          Entry entry = it.next();
-          if (entry.getEdit().isMetaEdit()) {
-            requiresReplication = true;
+    AtomicReference<Throwable> error = new AtomicReference<>();
+    AtomicInteger remainingTasks = new AtomicInteger(locs.size() - 1);
+    AtomicLong skippedEdits = new AtomicLong(0);
+
+    for (int i = 1, n = locs.size(); i < n; i++) {
+      final int replicaId = i;
+      FutureUtils.addListener(connection.replay(tableDesc.getTableName(),
+        locs.getRegionLocation(replicaId).getRegion().getEncodedNameAsBytes(), row, entries,
+        replicaId, numRetries, operationTimeoutNs), (r, e) -> {
+          if (e != null) {
+            LOG.warn("Failed to replicate to {}", locs.getRegionLocation(replicaId), e);
+            error.compareAndSet(null, e);
           } else {
-            it.remove();
-            skipEdits++;
+            AtomicUtils.updateMax(skippedEdits, r.longValue());
           }
-        }
-        skippedEdits.addAndGet(skipEdits);
-      }
-      return requiresReplication;
+          if (remainingTasks.decrementAndGet() == 0) {
+            if (error.get() != null) {
+              future.completeExceptionally(error.get());
+            } else {
+              future.complete(skippedEdits.get());
+            }
+          }
+        });
     }
   }
 
-  static class RegionReplicaSinkWriter extends SinkWriter {
-    RegionReplicaOutputSink sink;
-    ClusterConnection connection;
-    RpcControllerFactory rpcControllerFactory;
-    RpcRetryingCallerFactory rpcRetryingCallerFactory;
-    int operationTimeout;
-    ExecutorService pool;
-    Cache<TableName, Boolean> disabledAndDroppedTables;
-    TableDescriptors tableDescriptors;
-
-    public RegionReplicaSinkWriter(RegionReplicaOutputSink sink, ClusterConnection connection,
-        ExecutorService pool, int operationTimeout, TableDescriptors tableDescriptors) {
-      this.sink = sink;
-      this.connection = connection;
-      this.operationTimeout = operationTimeout;
-      this.rpcRetryingCallerFactory
-        = RpcRetryingCallerFactory.instantiate(connection.getConfiguration());
-      this.rpcControllerFactory = RpcControllerFactory.instantiate(connection.getConfiguration());
-      this.pool = pool;
-      this.tableDescriptors = tableDescriptors;
-
-      int nonExistentTableCacheExpiryMs = connection.getConfiguration()
-        .getInt("hbase.region.replica.replication.cache.disabledAndDroppedTables.expiryMs", 5000);
-      // A cache for non existing tables that have a default expiry of 5 sec. This means that if the
-      // table is created again with the same name, we might miss to replicate for that amount of
-      // time. But this cache prevents overloading meta requests for every edit from a deleted file.
-      disabledAndDroppedTables = CacheBuilder.newBuilder()
-        .expireAfterWrite(nonExistentTableCacheExpiryMs, TimeUnit.MILLISECONDS)
-        .initialCapacity(10)
-        .maximumSize(1000)
-        .build();
+  private void logSkipped(TableName tableName, List<Entry> entries, String reason) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Skipping {} entries because table {} is {}", entries.size(), tableName, reason);
+      for (Entry entry : entries) {
+        LOG.trace("Skipping : {}", entry);
+      }
     }
+  }
 
-    public void append(TableName tableName, byte[] encodedRegionName, byte[] row,
-        List<Entry> entries) throws IOException {
-
-      if (disabledAndDroppedTables.getIfPresent(tableName) != null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Skipping " + entries.size() + " entries because table " + tableName
-            + " is cached as a disabled or dropped table");
-          for (Entry entry : entries) {
-            LOG.trace("Skipping : " + entry);
-          }
-        }
-        sink.getSkippedEditsCounter().addAndGet(entries.size());
-        return;
+  private CompletableFuture<Long> replicate(TableDescriptor tableDesc, byte[] encodedRegionName,
+      List<Entry> entries) {
+    if (disabledTableCache.getIfPresent(tableDesc.getTableName()) != null) {
+      logSkipped(tableDesc.getTableName(), entries, "cached as a disabled table");
+      return CompletableFuture.completedFuture(Long.valueOf(entries.size()));
+    }
+    byte[] row = CellUtil.cloneRow(entries.get(0).getEdit().getCells().get(0));
+    CompletableFuture<RegionLocations> locateFuture = new CompletableFuture<>();
+    getRegionLocations(locateFuture, tableDesc, encodedRegionName, row, false);
+    CompletableFuture<Long> future = new CompletableFuture<>();
+    FutureUtils.addListener(locateFuture, (locs, error) -> {
+      if (error != null) {
+        future.completeExceptionally(error);
+      } else {
+        replicate(future, locs, tableDesc, encodedRegionName, row, entries);
       }
+    });
+    return future;
+  }
 
-      // If the table is disabled or dropped, we should not replay the entries, and we can skip
-      // replaying them. However, we might not know whether the table is disabled until we
-      // invalidate the cache and check from meta
-      RegionLocations locations = null;
-      boolean useCache = true;
-      while (true) {
-        // get the replicas of the primary region
+  @Override
+  public boolean replicate(ReplicateContext replicateContext) {
+    Map<byte[], Pair<TableDescriptor, List<Entry>>> encodedRegionName2Entries =
+      new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    long skippedEdits = 0;
+    RetryCounter retryCounter = retryCounterFactory.create();
+    outer: while (isRunning()) {
+      encodedRegionName2Entries.clear();
+      skippedEdits = 0;
+      for (Entry entry : replicateContext.getEntries()) {
+        Optional<TableDescriptor> tableDesc;
         try {
-          locations = RegionReplicaReplayCallable
-              .getRegionLocations(connection, tableName, row, useCache, 0);
-
-          if (locations == null) {
-            throw new HBaseIOException("Cannot locate locations for "
-                + tableName + ", row:" + Bytes.toStringBinary(row));
+          tableDesc = tableDescriptorCache.get(entry.getKey().getTableName());
+        } catch (ExecutionException e) {
+          LOG.warn("Failed to load table descriptor for {}, attempts={}",
+            entry.getKey().getTableName(), retryCounter.getAttemptTimes(), e.getCause());
+          if (!retryCounter.shouldRetry()) {
+            return false;
           }
-        } catch (TableNotFoundException e) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Skipping " + entries.size() + " entries because table " + tableName
-              + " is dropped. Adding table to cache.");
-            for (Entry entry : entries) {
-              LOG.trace("Skipping : " + entry);
-            }
+          try {
+            retryCounter.sleepUntilNextRetry();
+          } catch (InterruptedException e1) {
+            // restore the interrupted state
+            Thread.currentThread().interrupt();
+            return false;
           }
-          disabledAndDroppedTables.put(tableName, Boolean.TRUE); // put to cache. Value ignored
-          // skip this entry
-          sink.getSkippedEditsCounter().addAndGet(entries.size());
-          return;
+          continue outer;
         }
-
-        // check whether we should still replay this entry. If the regions are changed, or the
-        // entry is not coming from the primary region, filter it out.
-        HRegionLocation primaryLocation = locations.getDefaultRegionLocation();
-        if (!Bytes.equals(primaryLocation.getRegionInfo().getEncodedNameAsBytes(),
-          encodedRegionName)) {
-          if (useCache) {
-            useCache = false;
-            continue; // this will retry location lookup
-          }
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Skipping " + entries.size() + " entries in table " + tableName
-              + " because located region " + primaryLocation.getRegionInfo().getEncodedName()
-              + " is different than the original region " + Bytes.toStringBinary(encodedRegionName)
-              + " from WALEdit");
-            for (Entry entry : entries) {
-              LOG.trace("Skipping : " + entry);
-            }
-          }
-          sink.getSkippedEditsCounter().addAndGet(entries.size());
-          return;
+        if (!requiresReplication(tableDesc, entry)) {
+          skippedEdits++;
+          continue;
         }
-        break;
+        byte[] encodedRegionName = entry.getKey().getEncodedRegionName();
+        encodedRegionName2Entries
+          .computeIfAbsent(encodedRegionName, k -> Pair.newPair(tableDesc.get(), new ArrayList<>()))
+          .getSecond().add(entry);
       }
-
-      if (locations.size() == 1) {
-        return;
-      }
-
-      ArrayList<Future<ReplicateWALEntryResponse>> tasks = new ArrayList<>(locations.size() - 1);
-
-      // All passed entries should belong to one region because it is coming from the EntryBuffers
-      // split per region. But the regions might split and merge (unlike log recovery case).
-      for (int replicaId = 0; replicaId < locations.size(); replicaId++) {
-        HRegionLocation location = locations.getRegionLocation(replicaId);
-        if (!RegionReplicaUtil.isDefaultReplica(replicaId)) {
-          RegionInfo regionInfo = location == null
-              ? RegionReplicaUtil.getRegionInfoForReplica(
-                locations.getDefaultRegionLocation().getRegionInfo(), replicaId)
-              : location.getRegionInfo();
-          RegionReplicaReplayCallable callable = new RegionReplicaReplayCallable(connection,
-            rpcControllerFactory, tableName, location, regionInfo, row, entries,
-            sink.getSkippedEditsCounter());
-           Future<ReplicateWALEntryResponse> task = pool.submit(
-             new RetryingRpcCallable<>(rpcRetryingCallerFactory, callable, operationTimeout));
-           tasks.add(task);
-        }
+      break;
+    }
+    // send the request to regions
+    retryCounter = retryCounterFactory.create();
+    while (isRunning()) {
+      List<Pair<CompletableFuture<Long>, byte[]>> futureAndEncodedRegionNameList =
+        new ArrayList<Pair<CompletableFuture<Long>, byte[]>>();
+      for (Map.Entry<byte[], Pair<TableDescriptor, List<Entry>>> entry : encodedRegionName2Entries
+        .entrySet()) {
+        CompletableFuture<Long> future =
+          replicate(entry.getValue().getFirst(), entry.getKey(), entry.getValue().getSecond());
+        futureAndEncodedRegionNameList.add(Pair.newPair(future, entry.getKey()));
       }
-
-      boolean tasksCancelled = false;
-      for (int replicaId = 0; replicaId < tasks.size(); replicaId++) {
+      for (Pair<CompletableFuture<Long>, byte[]> pair : futureAndEncodedRegionNameList) {
+        byte[] encodedRegionName = pair.getSecond();
         try {
-          tasks.get(replicaId).get();
+          skippedEdits += pair.getFirst().get().longValue();
+          encodedRegionName2Entries.remove(encodedRegionName);
         } catch (InterruptedException e) {
-          throw new InterruptedIOException(e.getMessage());
+          // restore the interrupted state
+          Thread.currentThread().interrupt();
+          return false;
         } catch (ExecutionException e) {
+          Pair<TableDescriptor, List<Entry>> tableAndEntries =
+            encodedRegionName2Entries.get(encodedRegionName);
+          TableName tableName = tableAndEntries.getFirst().getTableName();
+          List<Entry> entries = tableAndEntries.getSecond();
           Throwable cause = e.getCause();
-          boolean canBeSkipped = false;
-          if (cause instanceof IOException) {
-            // The table can be disabled or dropped at this time. For disabled tables, we have no
-            // cheap mechanism to detect this case because meta does not contain this information.
-            // ClusterConnection.isTableDisabled() is a zk call which we cannot do for every replay
-            // RPC. So instead we start the replay RPC with retries and check whether the table is
-            // dropped or disabled which might cause SocketTimeoutException, or
-            // RetriesExhaustedException or similar if we get IOE.
-            if (cause instanceof TableNotFoundException
-                || connection.isTableDisabled(tableName)) {
-              disabledAndDroppedTables.put(tableName, Boolean.TRUE); // put to cache for later.
-              canBeSkipped = true;
-            } else if (tableDescriptors != null) {
-              TableDescriptor tableDescriptor = tableDescriptors.get(tableName);
-              if (tableDescriptor != null
-                  //(replicaId + 1) as no task is added for primary replica for replication
-                  && tableDescriptor.getRegionReplication() <= (replicaId + 1)) {
-                canBeSkipped = true;
-              }
-            }
-            if (canBeSkipped) {
-              if (LOG.isTraceEnabled()) {
-                LOG.trace("Skipping " + entries.size() + " entries in table " + tableName
-                    + " because received exception for dropped or disabled table",
-                  cause);
-                for (Entry entry : entries) {
-                  LOG.trace("Skipping : " + entry);
-                }
-              }
-              if (!tasksCancelled) {
-                sink.getSkippedEditsCounter().addAndGet(entries.size());
-                tasksCancelled = true; // so that we do not add to skipped counter again
-              }
-              continue;
-            }
-
-            // otherwise rethrow
-            throw (IOException)cause;
+          // The table can be disabled or dropped at this time. For disabled tables, we have no
+          // cheap mechanism to detect this case because meta does not contain this information.
+          // ClusterConnection.isTableDisabled() is a zk call which we cannot do for every replay
+          // RPC. So instead we start the replay RPC with retries and check whether the table is
+          // dropped or disabled which might cause SocketTimeoutException, or
+          // RetriesExhaustedException or similar if we get IOE.
+          if (cause instanceof TableNotFoundException) {
+            // add to cache that the table does not exist
+            tableDescriptorCache.put(tableName, Optional.empty());
+            logSkipped(tableName, entries, "dropped");
+            skippedEdits += entries.size();
+            encodedRegionName2Entries.remove(encodedRegionName);
+            continue;
+          }
+          boolean disabled = false;
+          try {
+            disabled = connection.getAdmin().isTableDisabled(tableName).get();
+          } catch (InterruptedException e1) {
+            // restore the interrupted state
+            Thread.currentThread().interrupt();
+            return false;
+          } catch (ExecutionException e1) {
+            LOG.warn("Failed to test whether {} is disabled, assume it is not disabled", tableName,
+              e1.getCause());
+          }
+          if (disabled) {
+            disabledTableCache.put(tableName, tableName);
+            logSkipped(tableName, entries, "disabled");
+            skippedEdits += entries.size();
+            encodedRegionName2Entries.remove(encodedRegionName);
+            continue;
           }
-          // unexpected exception
-          throw new IOException(cause);
+          LOG.warn("Failed to replicate {} entries for region {} of table {}", entries.size(),
+            Bytes.toStringBinary(encodedRegionName), tableName);
+        }
+      }
+      // we have done
+      if (encodedRegionName2Entries.isEmpty()) {
+        ctx.getMetrics().incrLogEditsFiltered(skippedEdits);
+        return true;
+      } else {
+        LOG.warn("Failed to replicate all entris, retry={}", retryCounter.getAttemptTimes());
+        if (!retryCounter.shouldRetry()) {
+          return false;
+        }
+        try {
+          retryCounter.sleepUntilNextRetry();
+        } catch (InterruptedException e) {
+          // restore the interrupted state
+          Thread.currentThread().interrupt();
+          return false;
         }
       }
     }
-  }
 
-  static class RetryingRpcCallable<V> implements Callable<V> {
-    RpcRetryingCallerFactory factory;
-    RetryingCallable<V> callable;
-    int timeout;
-    public RetryingRpcCallable(RpcRetryingCallerFactory factory, RetryingCallable<V> callable,
-        int timeout) {
-      this.factory = factory;
-      this.callable = callable;
-      this.timeout = timeout;
-    }
-    @Override
-    public V call() throws Exception {
-      return factory.<V>newCaller().callWithRetries(callable, timeout);
-    }
+    return false;
   }
 
-  /**
-   * Calls replay on the passed edits for the given set of entries belonging to the region. It skips
-   * the entry if the region boundaries have changed or the region is gone.
-   */
-  static class RegionReplicaReplayCallable extends
-      RegionAdminServiceCallable<ReplicateWALEntryResponse> {
-    private final List<Entry> entries;
-    private final byte[] initialEncodedRegionName;
-    private final AtomicLong skippedEntries;
-
-    public RegionReplicaReplayCallable(ClusterConnection connection,
-        RpcControllerFactory rpcControllerFactory, TableName tableName,
-        HRegionLocation location, RegionInfo regionInfo, byte[] row,List<Entry> entries,
-        AtomicLong skippedEntries) {
-      super(connection, rpcControllerFactory, location, tableName, row, regionInfo.getReplicaId());
-      this.entries = entries;
-      this.skippedEntries = skippedEntries;
-      this.initialEncodedRegionName = regionInfo.getEncodedNameAsBytes();
-    }
-
-    @Override
-    public ReplicateWALEntryResponse call(HBaseRpcController controller) throws Exception {
-      // Check whether we should still replay this entry. If the regions are changed, or the
-      // entry is not coming form the primary region, filter it out because we do not need it.
-      // Regions can change because of (1) region split (2) region merge (3) table recreated
-      boolean skip = false;
-      if (!Bytes.equals(location.getRegionInfo().getEncodedNameAsBytes(),
-          initialEncodedRegionName)) {
-        skip = true;
-      }
-      if (!this.entries.isEmpty() && !skip) {
-        Entry[] entriesArray = new Entry[this.entries.size()];
-        entriesArray = this.entries.toArray(entriesArray);
-
-        // set the region name for the target region replica
-        Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
-            ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray, location
-                .getRegionInfo().getEncodedNameAsBytes(), null, null, null);
-        controller.setCellScanner(p.getSecond());
-        return stub.replay(controller, p.getFirst());
-      }
+  @Override
+  public boolean canReplicateToSameCluster() {
+    return true;
+  }
 
-      if (skip) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Skipping " + entries.size() + " entries in table " + tableName
-            + " because located region " + location.getRegionInfo().getEncodedName()
-            + " is different than the original region "
-            + Bytes.toStringBinary(initialEncodedRegionName) + " from WALEdit");
-          for (Entry entry : entries) {
-            LOG.trace("Skipping : " + entry);
-          }
-        }
-        skippedEntries.addAndGet(entries.size());
-      }
-      return ReplicateWALEntryResponse.newBuilder().build();
-    }
+  @Override
+  protected WALEntryFilter getScopeWALEntryFilter() {
+    // we do not care about scope. We replicate everything.
+    return null;
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 8e001e6..b58fce3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -283,7 +283,7 @@ public class ReplicationSource implements ReplicationSourceInterface {
       tableDescriptors = ((HRegionServer) server).getTableDescriptors();
     }
     replicationEndpoint
-      .init(new ReplicationEndpoint.Context(conf, replicationPeer.getConfiguration(), fs,
+      .init(new ReplicationEndpoint.Context(server, conf, replicationPeer.getConfiguration(), fs,
         replicationPeer.getId(), clusterId, replicationPeer, metrics, tableDescriptors, server));
     replicationEndpoint.start();
     replicationEndpoint.awaitRunning(waitOnEndpointSeconds, TimeUnit.SECONDS);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
index 04db81a..017d7c9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
@@ -20,16 +20,17 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.UUID;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.Cell.Type;
 import org.apache.hadoop.hbase.CellBuilderFactory;
@@ -42,7 +43,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionLocator;
@@ -51,12 +51,12 @@ import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.testclassification.FlakeyTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -383,9 +383,8 @@ public class TestRegionReplicaReplicationEndpoint {
     testRegionReplicaReplicationIgnores(false, true);
   }
 
-  public void testRegionReplicaReplicationIgnores(boolean dropTable, boolean disableReplication)
+  private void testRegionReplicaReplicationIgnores(boolean dropTable, boolean disableReplication)
       throws Exception {
-
     // tests having edits from a disabled or dropped table is handled correctly by skipping those
     // entries and further edits after the edits from dropped/disabled table can be replicated
     // without problems.
@@ -405,8 +404,7 @@ public class TestRegionReplicaReplicationEndpoint {
     HTU.getAdmin().createTable(htd);
 
     // both tables are created, now pause replication
-    ReplicationAdmin admin = new ReplicationAdmin(HTU.getConfiguration());
-    admin.disablePeer(ServerRegionReplicaUtil.getReplicationPeerId());
+    HTU.getAdmin().disableReplicationPeer(ServerRegionReplicaUtil.getReplicationPeerId());
 
     // now that the replication is disabled, write to the table to be dropped, then drop the table.
 
@@ -416,19 +414,9 @@ public class TestRegionReplicaReplicationEndpoint {
 
     HTU.loadNumericRows(tableToBeDisabled, HBaseTestingUtility.fam1, 6000, 7000);
 
-    AtomicLong skippedEdits = new AtomicLong();
-    RegionReplicaReplicationEndpoint.RegionReplicaOutputSink sink =
-        mock(RegionReplicaReplicationEndpoint.RegionReplicaOutputSink.class);
-    when(sink.getSkippedEditsCounter()).thenReturn(skippedEdits);
-    FSTableDescriptors fstd = new FSTableDescriptors(HTU.getConfiguration(),
-        FileSystem.get(HTU.getConfiguration()), HTU.getDefaultRootDirPath());
-    RegionReplicaReplicationEndpoint.RegionReplicaSinkWriter sinkWriter =
-        new RegionReplicaReplicationEndpoint.RegionReplicaSinkWriter(sink,
-            (ClusterConnection) connection, Executors.newSingleThreadExecutor(), Integer.MAX_VALUE,
-            fstd);
     RegionLocator rl = connection.getRegionLocator(toBeDisabledTable);
     HRegionLocation hrl = rl.getRegionLocation(HConstants.EMPTY_BYTE_ARRAY);
-    byte[] encodedRegionName = hrl.getRegionInfo().getEncodedNameAsBytes();
+    byte[] encodedRegionName = hrl.getRegion().getEncodedNameAsBytes();
 
     Cell cell = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(Bytes.toBytes("A"))
         .setFamily(HTU.fam1).setValue(Bytes.toBytes("VAL")).setType(Type.Put).build();
@@ -436,7 +424,6 @@ public class TestRegionReplicaReplicationEndpoint {
       new WALKeyImpl(encodedRegionName, toBeDisabledTable, 1),
         new WALEdit()
             .add(cell));
-
     HTU.getAdmin().disableTable(toBeDisabledTable); // disable the table
     if (dropTable) {
       HTU.getAdmin().deleteTable(toBeDisabledTable);
@@ -445,11 +432,23 @@ public class TestRegionReplicaReplicationEndpoint {
       HTU.getAdmin().modifyTable(toBeDisabledTable, htd);
       HTU.getAdmin().enableTable(toBeDisabledTable);
     }
-    sinkWriter.append(toBeDisabledTable, encodedRegionName,
-      HConstants.EMPTY_BYTE_ARRAY, Lists.newArrayList(entry, entry));
-
-    assertEquals(2, skippedEdits.get());
 
+    HRegionServer rs = HTU.getMiniHBaseCluster().getRegionServer(0);
+    MetricsSource metrics = mock(MetricsSource.class);
+    ReplicationEndpoint.Context ctx =
+      new ReplicationEndpoint.Context(rs, HTU.getConfiguration(), HTU.getConfiguration(),
+        HTU.getTestFileSystem(), ServerRegionReplicaUtil.getReplicationPeerId(),
+        UUID.fromString(rs.getClusterId()), rs.getReplicationSourceService().getReplicationPeers()
+          .getPeer(ServerRegionReplicaUtil.getReplicationPeerId()),
+        metrics, rs.getTableDescriptors(), rs);
+    RegionReplicaReplicationEndpoint rrpe = new RegionReplicaReplicationEndpoint();
+    rrpe.init(ctx);
+    rrpe.start();
+    ReplicationEndpoint.ReplicateContext repCtx = new ReplicationEndpoint.ReplicateContext();
+    repCtx.setEntries(Lists.newArrayList(entry, entry));
+    assertTrue(rrpe.replicate(repCtx));
+    verify(metrics, times(1)).incrLogEditsFiltered(eq(2L));
+    rrpe.stop();
     if (disableReplication) {
       // enable replication again so that we can verify replication
       HTU.getAdmin().disableTable(toBeDisabledTable); // disable the table
@@ -460,17 +459,14 @@ public class TestRegionReplicaReplicationEndpoint {
 
     try {
       // load some data to the to-be-dropped table
-
       // load the data to the table
       HTU.loadNumericRows(table, HBaseTestingUtility.fam1, 0, 1000);
 
       // now enable the replication
-      admin.enablePeer(ServerRegionReplicaUtil.getReplicationPeerId());
+      HTU.getAdmin().enableReplicationPeer(ServerRegionReplicaUtil.getReplicationPeerId());
 
       verifyReplication(tableName, regionReplication, 0, 1000);
-
     } finally {
-      admin.close();
       table.close();
       rl.close();
       tableToBeDisabled.close();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
index 1e4a49e..0ec7d54 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
@@ -19,15 +19,16 @@ package org.apache.hadoop.hbase.replication.regionserver;
 
 import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.closeRegion;
 import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.openRegion;
-import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Optional;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -37,24 +38,22 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.WALCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.WALCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.WALObserver;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint.ReplicateContext;
-import org.apache.hadoop.hbase.replication.regionserver.RegionReplicaReplicationEndpoint.RegionReplicaReplayCallable;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -74,8 +73,6 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
-
 /**
  * Tests RegionReplicaReplicationEndpoint. Unlike TestRegionReplicaReplicationEndpoint this
  * class contains lower level tests using callables.
@@ -179,39 +176,34 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
   public void testReplayCallable() throws Exception {
     // tests replaying the edits to a secondary region replica using the Callable directly
     openRegion(HTU, rs0, hriSecondary);
-    ClusterConnection connection =
-        (ClusterConnection) ConnectionFactory.createConnection(HTU.getConfiguration());
 
-    //load some data to primary
+    // load some data to primary
     HTU.loadNumericRows(table, f, 0, 1000);
 
     Assert.assertEquals(1000, entries.size());
-    // replay the edits to the secondary using replay callable
-    replicateUsingCallable(connection, entries);
+    try (AsyncClusterConnection conn = ClusterConnectionFactory
+      .createAsyncClusterConnection(HTU.getConfiguration(), null, User.getCurrent())) {
+      // replay the edits to the secondary using replay callable
+      replicateUsingCallable(conn, entries);
+    }
 
     Region region = rs0.getRegion(hriSecondary.getEncodedName());
     HTU.verifyNumericRows(region, f, 0, 1000);
 
     HTU.deleteNumericRows(table, f, 0, 1000);
     closeRegion(HTU, rs0, hriSecondary);
-    connection.close();
   }
 
-  private void replicateUsingCallable(ClusterConnection connection, Queue<Entry> entries)
-      throws IOException, RuntimeException {
+  private void replicateUsingCallable(AsyncClusterConnection connection, Queue<Entry> entries)
+      throws IOException, ExecutionException, InterruptedException {
     Entry entry;
     while ((entry = entries.poll()) != null) {
       byte[] row = CellUtil.cloneRow(entry.getEdit().getCells().get(0));
-      RegionLocations locations = connection.locateRegion(tableName, row, true, true);
-      RegionReplicaReplayCallable callable = new RegionReplicaReplayCallable(connection,
-        RpcControllerFactory.instantiate(connection.getConfiguration()),
-        table.getName(), locations.getRegionLocation(1),
-        locations.getRegionLocation(1).getRegionInfo(), row, Lists.newArrayList(entry),
-        new AtomicLong());
-
-      RpcRetryingCallerFactory factory = RpcRetryingCallerFactory.instantiate(
-        connection.getConfiguration());
-      factory.<ReplicateWALEntryResponse> newCaller().callWithRetries(callable, 10000);
+      RegionLocations locations = connection.getRegionLocations(tableName, row, true).get();
+      connection
+        .replay(tableName, locations.getRegionLocation(1).getRegion().getEncodedNameAsBytes(), row,
+          Collections.singletonList(entry), 1, Integer.MAX_VALUE, TimeUnit.SECONDS.toNanos(10))
+        .get();
     }
   }
 
@@ -219,49 +211,49 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
   public void testReplayCallableWithRegionMove() throws Exception {
     // tests replaying the edits to a secondary region replica using the Callable directly while
     // the region is moved to another location.It tests handling of RME.
-    openRegion(HTU, rs0, hriSecondary);
-    ClusterConnection connection =
-        (ClusterConnection) ConnectionFactory.createConnection(HTU.getConfiguration());
-    //load some data to primary
-    HTU.loadNumericRows(table, f, 0, 1000);
+    try (AsyncClusterConnection conn = ClusterConnectionFactory
+      .createAsyncClusterConnection(HTU.getConfiguration(), null, User.getCurrent())) {
+      openRegion(HTU, rs0, hriSecondary);
+      // load some data to primary
+      HTU.loadNumericRows(table, f, 0, 1000);
 
-    Assert.assertEquals(1000, entries.size());
-    // replay the edits to the secondary using replay callable
-    replicateUsingCallable(connection, entries);
+      Assert.assertEquals(1000, entries.size());
 
-    Region region = rs0.getRegion(hriSecondary.getEncodedName());
-    HTU.verifyNumericRows(region, f, 0, 1000);
+      // replay the edits to the secondary using replay callable
+      replicateUsingCallable(conn, entries);
 
-    HTU.loadNumericRows(table, f, 1000, 2000); // load some more data to primary
+      Region region = rs0.getRegion(hriSecondary.getEncodedName());
+      HTU.verifyNumericRows(region, f, 0, 1000);
 
-    // move the secondary region from RS0 to RS1
-    closeRegion(HTU, rs0, hriSecondary);
-    openRegion(HTU, rs1, hriSecondary);
+      HTU.loadNumericRows(table, f, 1000, 2000); // load some more data to primary
 
-    // replicate the new data
-    replicateUsingCallable(connection, entries);
+      // move the secondary region from RS0 to RS1
+      closeRegion(HTU, rs0, hriSecondary);
+      openRegion(HTU, rs1, hriSecondary);
 
-    region = rs1.getRegion(hriSecondary.getEncodedName());
-    // verify the new data. old data may or may not be there
-    HTU.verifyNumericRows(region, f, 1000, 2000);
+      // replicate the new data
+      replicateUsingCallable(conn, entries);
 
-    HTU.deleteNumericRows(table, f, 0, 2000);
-    closeRegion(HTU, rs1, hriSecondary);
-    connection.close();
+      region = rs1.getRegion(hriSecondary.getEncodedName());
+      // verify the new data. old data may or may not be there
+      HTU.verifyNumericRows(region, f, 1000, 2000);
+
+      HTU.deleteNumericRows(table, f, 0, 2000);
+      closeRegion(HTU, rs1, hriSecondary);
+    }
   }
 
   @Test
   public void testRegionReplicaReplicationEndpointReplicate() throws Exception {
     // tests replaying the edits to a secondary region replica using the RRRE.replicate()
     openRegion(HTU, rs0, hriSecondary);
-    ClusterConnection connection =
-        (ClusterConnection) ConnectionFactory.createConnection(HTU.getConfiguration());
     RegionReplicaReplicationEndpoint replicator = new RegionReplicaReplicationEndpoint();
 
     ReplicationEndpoint.Context context = mock(ReplicationEndpoint.Context.class);
     when(context.getConfiguration()).thenReturn(HTU.getConfiguration());
     when(context.getMetrics()).thenReturn(mock(MetricsSource.class));
-
+    when(context.getServer()).thenReturn(rs0);
+    when(context.getTableDescriptors()).thenReturn(rs0.getTableDescriptors());
     replicator.init(context);
     replicator.startAsync();
 
@@ -273,12 +265,11 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
     final String fakeWalGroupId = "fakeWALGroup";
     replicator.replicate(new ReplicateContext().setEntries(Lists.newArrayList(entries))
         .setWalGroupId(fakeWalGroupId));
-
+    replicator.stop();
     Region region = rs0.getRegion(hriSecondary.getEncodedName());
     HTU.verifyNumericRows(region, f, 0, 1000);
 
     HTU.deleteNumericRows(table, f, 0, 1000);
     closeRegion(HTU, rs0, hriSecondary);
-    connection.close();
   }
 }


[hbase] 03/11: HBASE-21526 Use AsyncClusterConnection in ServerManager for getRsAdmin

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

zhangduo pushed a commit to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 1ee798ae0364e6aee8de8b05f686a74075a3d0cb
Author: zhangduo <zh...@apache.org>
AuthorDate: Thu Dec 6 21:25:34 2018 +0800

    HBASE-21526 Use AsyncClusterConnection in ServerManager for getRsAdmin
---
 .../hbase/client/AsyncClusterConnection.java       |   6 +
 .../hadoop/hbase/client/AsyncConnectionImpl.java   |   5 +
 .../hbase/client/AsyncRegionServerAdmin.java       | 210 +++++++++++++++++++++
 .../org/apache/hadoop/hbase/util/FutureUtils.java  |   2 +-
 .../org/apache/hadoop/hbase/master/HMaster.java    |  15 +-
 .../apache/hadoop/hbase/master/ServerManager.java  |  67 -------
 .../master/procedure/RSProcedureDispatcher.java    |  19 +-
 7 files changed, 241 insertions(+), 83 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
index c7dea25..1327fd7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -27,6 +28,11 @@ import org.apache.yetus.audience.InterfaceAudience;
 public interface AsyncClusterConnection extends AsyncConnection {
 
   /**
+   * Get the admin service for the given region server.
+   */
+  AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName);
+
+  /**
    * Get the nonce generator for this connection.
    */
   NonceGenerator getNonceGenerator();
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 f8ad97a..baeb1ef 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
@@ -377,4 +377,9 @@ class AsyncConnectionImpl implements AsyncClusterConnection {
   public void clearRegionLocationCache() {
     locator.clearCache();
   }
+
+  @Override
+  public AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName) {
+    return new AsyncRegionServerAdmin(serverName, this);
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
new file mode 100644
index 0000000..9accd89
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
@@ -0,0 +1,210 @@
+/**
+ * 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 java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
+
+/**
+ * A simple wrapper of the {@link AdminService} for a region server, which returns a
+ * {@link CompletableFuture}. This is easier to use, as if you use the raw protobuf interface, you
+ * need to get the result from the {@link RpcCallback}, and if there is an exception, you need to
+ * get it from the {@link RpcController} passed in.
+ * <p/>
+ * Notice that there is no retry, and this is intentional. We have different retry for different
+ * usage for now, if later we want to unify them, we can move the retry logic into this class.
+ */
+@InterfaceAudience.Private
+public class AsyncRegionServerAdmin {
+
+  private final ServerName server;
+
+  private final AsyncConnectionImpl conn;
+
+  AsyncRegionServerAdmin(ServerName server, AsyncConnectionImpl conn) {
+    this.server = server;
+    this.conn = conn;
+  }
+
+  @FunctionalInterface
+  private interface RpcCall<RESP> {
+    void call(AdminService.Interface stub, HBaseRpcController controller, RpcCallback<RESP> done);
+  }
+
+  private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall) {
+    CompletableFuture<RESP> future = new CompletableFuture<>();
+    HBaseRpcController controller = conn.rpcControllerFactory.newController();
+    try {
+      rpcCall.call(conn.getAdminStub(server), controller, new RpcCallback<RESP>() {
+
+        @Override
+        public void run(RESP resp) {
+          if (controller.failed()) {
+            future.completeExceptionally(controller.getFailed());
+          } else {
+            future.complete(resp);
+          }
+        }
+      });
+    } catch (IOException e) {
+      future.completeExceptionally(e);
+    }
+    return future;
+  }
+
+  public CompletableFuture<GetRegionInfoResponse> getRegionInfo(GetRegionInfoRequest request) {
+    return call((stub, controller, done) -> stub.getRegionInfo(controller, request, done));
+  }
+
+  public CompletableFuture<GetStoreFileResponse> getStoreFile(GetStoreFileRequest request) {
+    return call((stub, controller, done) -> stub.getStoreFile(controller, request, done));
+  }
+
+  public CompletableFuture<GetOnlineRegionResponse> getOnlineRegion(
+      GetOnlineRegionRequest request) {
+    return call((stub, controller, done) -> stub.getOnlineRegion(controller, request, done));
+  }
+
+  public CompletableFuture<OpenRegionResponse> openRegion(OpenRegionRequest request) {
+    return call((stub, controller, done) -> stub.openRegion(controller, request, done));
+  }
+
+  public CompletableFuture<WarmupRegionResponse> warmupRegion(WarmupRegionRequest request) {
+    return call((stub, controller, done) -> stub.warmupRegion(controller, request, done));
+  }
+
+  public CompletableFuture<CloseRegionResponse> closeRegion(CloseRegionRequest request) {
+    return call((stub, controller, done) -> stub.closeRegion(controller, request, done));
+  }
+
+  public CompletableFuture<FlushRegionResponse> flushRegion(FlushRegionRequest request) {
+    return call((stub, controller, done) -> stub.flushRegion(controller, request, done));
+  }
+
+  public CompletableFuture<CompactionSwitchResponse> compactionSwitch(
+      CompactionSwitchRequest request) {
+    return call((stub, controller, done) -> stub.compactionSwitch(controller, request, done));
+  }
+
+  public CompletableFuture<CompactRegionResponse> compactRegion(CompactRegionRequest request) {
+    return call((stub, controller, done) -> stub.compactRegion(controller, request, done));
+  }
+
+  public CompletableFuture<ReplicateWALEntryResponse> replicateWALEntry(
+      ReplicateWALEntryRequest request) {
+    return call((stub, controller, done) -> stub.replicateWALEntry(controller, request, done));
+  }
+
+  public CompletableFuture<ReplicateWALEntryResponse> replay(ReplicateWALEntryRequest request) {
+    return call((stub, controller, done) -> stub.replay(controller, request, done));
+  }
+
+  public CompletableFuture<RollWALWriterResponse> rollWALWriter(RollWALWriterRequest request) {
+    return call((stub, controller, done) -> stub.rollWALWriter(controller, request, done));
+  }
+
+  public CompletableFuture<GetServerInfoResponse> getServerInfo(GetServerInfoRequest request) {
+    return call((stub, controller, done) -> stub.getServerInfo(controller, request, done));
+  }
+
+  public CompletableFuture<StopServerResponse> stopServer(StopServerRequest request) {
+    return call((stub, controller, done) -> stub.stopServer(controller, request, done));
+  }
+
+  public CompletableFuture<UpdateFavoredNodesResponse> updateFavoredNodes(
+      UpdateFavoredNodesRequest request) {
+    return call((stub, controller, done) -> stub.updateFavoredNodes(controller, request, done));
+  }
+
+  public CompletableFuture<UpdateConfigurationResponse> updateConfiguration(
+      UpdateConfigurationRequest request) {
+    return call((stub, controller, done) -> stub.updateConfiguration(controller, request, done));
+  }
+
+  public CompletableFuture<GetRegionLoadResponse> getRegionLoad(GetRegionLoadRequest request) {
+    return call((stub, controller, done) -> stub.getRegionLoad(controller, request, done));
+  }
+
+  public CompletableFuture<ClearCompactionQueuesResponse> clearCompactionQueues(
+      ClearCompactionQueuesRequest request) {
+    return call((stub, controller, done) -> stub.clearCompactionQueues(controller, request, done));
+  }
+
+  public CompletableFuture<ClearRegionBlockCacheResponse> clearRegionBlockCache(
+      ClearRegionBlockCacheRequest request) {
+    return call((stub, controller, done) -> stub.clearRegionBlockCache(controller, request, done));
+  }
+
+  public CompletableFuture<GetSpaceQuotaSnapshotsResponse> getSpaceQuotaSnapshots(
+      GetSpaceQuotaSnapshotsRequest request) {
+    return call((stub, controller, done) -> stub.getSpaceQuotaSnapshots(controller, request, done));
+  }
+
+  public CompletableFuture<ExecuteProceduresResponse> executeProcedures(
+      ExecuteProceduresRequest request) {
+    return call((stub, controller, done) -> stub.executeProcedures(controller, request, done));
+  }
+}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
index 6f0077c..bdcb52c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
@@ -143,4 +143,4 @@ public final class FutureUtils {
     future.completeExceptionally(e);
     return future;
   }
-}
\ No newline at end of file
+}
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 8aa63fe..b577d13 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
@@ -195,6 +195,7 @@ import org.apache.hadoop.hbase.util.BloomFilterUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.EncryptionTest;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.HasThread;
@@ -227,6 +228,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
@@ -1955,6 +1957,15 @@ public class HMaster extends HRegionServer implements MasterServices {
     });
   }
 
+  private void warmUpRegion(ServerName server, RegionInfo region) {
+    FutureUtils.addListener(asyncClusterConnection.getRegionServerAdmin(server)
+      .warmupRegion(RequestConverter.buildWarmupRegionRequest(region)), (r, e) -> {
+        if (e != null) {
+          LOG.warn("Failed to warm up region {} on server {}", region, server, e);
+        }
+      });
+  }
+
   // Public so can be accessed by tests. Blocks until move is done.
   // Replace with an async implementation from which you can get
   // a success/failure result.
@@ -2026,7 +2037,9 @@ public class HMaster extends HRegionServer implements MasterServices {
       // Warmup the region on the destination before initiating the move. this call
       // is synchronous and takes some time. doing it before the source region gets
       // closed
-      serverManager.sendRegionWarmup(rp.getDestination(), hri);
+      // A region server could reject the close request because it either does not
+      // have the specified region or the region is being split.
+      warmUpRegion(rp.getDestination(), hri);
 
       LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
       Future<byte []> future = this.assignmentManager.moveAsync(rp);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 0fb1551..a8d5e21 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -51,12 +50,9 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -159,25 +155,16 @@ public class ServerManager {
   private final ConcurrentNavigableMap<ServerName, ServerMetrics> onlineServers =
     new ConcurrentSkipListMap<>();
 
-  /**
-   * Map of admin interfaces per registered regionserver; these interfaces we use to control
-   * regionservers out on the cluster
-   */
-  private final Map<ServerName, AdminService.BlockingInterface> rsAdmins = new HashMap<>();
-
   /** List of region servers that should not get any more new regions. */
   private final ArrayList<ServerName> drainingServers = new ArrayList<>();
 
   private final MasterServices master;
-  private final ClusterConnection connection;
 
   private final DeadServer deadservers = new DeadServer();
 
   private final long maxSkew;
   private final long warningSkew;
 
-  private final RpcControllerFactory rpcControllerFactory;
-
   /** Listeners that are called on server events. */
   private List<ServerListener> listeners = new CopyOnWriteArrayList<>();
 
@@ -189,8 +176,6 @@ public class ServerManager {
     Configuration c = master.getConfiguration();
     maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
     warningSkew = c.getLong("hbase.master.warningclockskew", 10000);
-    this.connection = master.getClusterConnection();
-    this.rpcControllerFactory = this.connection == null? null: connection.getRpcControllerFactory();
     persistFlushedSequenceId = c.getBoolean(PERSIST_FLUSHEDSEQUENCEID,
         PERSIST_FLUSHEDSEQUENCEID_DEFAULT);
   }
@@ -438,7 +423,6 @@ public class ServerManager {
   void recordNewServerWithLock(final ServerName serverName, final ServerMetrics sl) {
     LOG.info("Registering regionserver=" + serverName);
     this.onlineServers.put(serverName, sl);
-    this.rsAdmins.remove(serverName);
   }
 
   @VisibleForTesting
@@ -633,7 +617,6 @@ public class ServerManager {
       this.onlineServers.remove(sn);
       onlineServers.notifyAll();
     }
-    this.rsAdmins.remove(sn);
   }
 
   /*
@@ -676,34 +659,6 @@ public class ServerManager {
     return this.drainingServers.add(sn);
   }
 
-  // RPC methods to region servers
-
-  private HBaseRpcController newRpcController() {
-    return rpcControllerFactory == null ? null : rpcControllerFactory.newController();
-  }
-
-  /**
-   * Sends a WARMUP RPC to the specified server to warmup the specified region.
-   * <p>
-   * A region server could reject the close request because it either does not
-   * have the specified region or the region is being split.
-   * @param server server to warmup a region
-   * @param region region to  warmup
-   */
-  public void sendRegionWarmup(ServerName server,
-      RegionInfo region) {
-    if (server == null) return;
-    try {
-      AdminService.BlockingInterface admin = getRsAdmin(server);
-      HBaseRpcController controller = newRpcController();
-      ProtobufUtil.warmupRegion(controller, admin, region);
-    } catch (IOException e) {
-      LOG.error("Received exception in RPC for warmup server:" +
-        server + "region: " + region +
-        "exception: " + e);
-    }
-  }
-
   /**
    * Contacts a region server and waits up to timeout ms
    * to close the region.  This bypasses the active hmaster.
@@ -737,28 +692,6 @@ public class ServerManager {
   }
 
   /**
-   * @param sn
-   * @return Admin interface for the remote regionserver named <code>sn</code>
-   * @throws IOException
-   * @throws RetriesExhaustedException wrapping a ConnectException if failed
-   */
-  public AdminService.BlockingInterface getRsAdmin(final ServerName sn)
-  throws IOException {
-    AdminService.BlockingInterface admin = this.rsAdmins.get(sn);
-    if (admin == null) {
-      LOG.debug("New admin connection to " + sn.toString());
-      if (sn.equals(master.getServerName()) && master instanceof HRegionServer) {
-        // A master is also a region server now, see HBASE-10569 for details
-        admin = ((HRegionServer)master).getRSRpcServices();
-      } else {
-        admin = this.connection.getAdmin(sn);
-      }
-      this.rsAdmins.put(sn, admin);
-    }
-    return admin;
-  }
-
-  /**
    * Calculate min necessary to start. This is not an absolute. It is just
    * a friction that will cause us hang around a bit longer waiting on
    * RegionServers to check-in.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index b8ba7b3..fff64e3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -25,6 +25,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.hbase.CallQueueTooBigException;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.MasterServices;
@@ -33,6 +34,7 @@ import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher;
 import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -41,11 +43,9 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
 import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
@@ -219,13 +219,8 @@ public class RSProcedureDispatcher
       this.remoteProcedures = remoteProcedures;
     }
 
-    private AdminService.BlockingInterface getRsAdmin() throws IOException {
-      final AdminService.BlockingInterface admin = master.getServerManager().getRsAdmin(serverName);
-      if (admin == null) {
-        throw new IOException("Attempting to send OPEN RPC to server " + getServerName() +
-          " failed because no RPC connection found to this server");
-      }
-      return admin;
+    private AsyncRegionServerAdmin  getRsAdmin() throws IOException {
+      return master.getAsyncClusterConnection().getRegionServerAdmin(serverName);
     }
 
     protected final ServerName getServerName() {
@@ -345,11 +340,7 @@ public class RSProcedureDispatcher
     @VisibleForTesting
     protected ExecuteProceduresResponse sendRequest(final ServerName serverName,
         final ExecuteProceduresRequest request) throws IOException {
-      try {
-        return getRsAdmin().executeProcedures(null, request);
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
-      }
+      return FutureUtils.get(getRsAdmin().executeProcedures(request));
     }
 
     protected final void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {


[hbase] 01/11: HBASE-21515 Also initialize an AsyncClusterConnection in HRegionServer

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

zhangduo pushed a commit to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 713da50410d86530e027a9ee76764f2d6abe6952
Author: zhangduo <zh...@apache.org>
AuthorDate: Fri Nov 30 08:23:47 2018 +0800

    HBASE-21515 Also initialize an AsyncClusterConnection in HRegionServer
---
 .../hbase/client/AsyncClusterConnection.java       | 38 +++++++++++++
 .../hadoop/hbase/client/AsyncConnectionImpl.java   | 14 +++--
 .../hbase/client/ClusterConnectionFactory.java     | 63 ++++++++++++++++++++++
 .../hadoop/hbase/client/ConnectionFactory.java     |  2 +-
 .../hbase/client/TestAsyncAdminRpcPriority.java    |  2 +-
 .../hbase/client/TestAsyncTableRpcPriority.java    |  2 +-
 .../apache/hadoop/hbase/util/ReflectionUtils.java  | 22 ++++----
 .../main/java/org/apache/hadoop/hbase/Server.java  | 20 +++++++
 .../org/apache/hadoop/hbase/master/HMaster.java    |  3 ++
 .../hadoop/hbase/regionserver/HRegionServer.java   | 56 +++++++++++++------
 .../regionserver/ReplicationSyncUp.java            |  6 +++
 .../hadoop/hbase/MockRegionServerServices.java     |  5 ++
 .../client/TestAsyncNonMetaRegionLocator.java      |  2 +-
 ...stAsyncNonMetaRegionLocatorConcurrenyLimit.java |  2 +-
 .../hbase/client/TestAsyncRegionLocator.java       |  2 +-
 .../TestAsyncSingleRequestRpcRetryingCaller.java   |  4 +-
 .../hbase/master/MockNoopMasterServices.java       |  6 +++
 .../hadoop/hbase/master/MockRegionServer.java      |  5 ++
 .../hbase/master/TestActiveMasterManager.java      |  6 +++
 .../hbase/master/cleaner/TestHFileCleaner.java     |  6 +++
 .../hbase/master/cleaner/TestHFileLinkCleaner.java |  6 +++
 .../hbase/master/cleaner/TestLogsCleaner.java      |  6 +++
 .../cleaner/TestReplicationHFileCleaner.java       |  6 +++
 .../hbase/regionserver/TestHeapMemoryManager.java  |  6 +++
 .../hbase/regionserver/TestSplitLogWorker.java     |  6 +++
 .../hadoop/hbase/regionserver/TestWALLockup.java   |  6 +++
 .../replication/TestReplicationTrackerZKImpl.java  |  6 +++
 .../regionserver/TestReplicationSourceManager.java |  6 +++
 .../security/token/TestTokenAuthentication.java    |  6 +++
 .../org/apache/hadoop/hbase/util/MockServer.java   |  6 +++
 30 files changed, 290 insertions(+), 36 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
new file mode 100644
index 0000000..c7dea25
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
@@ -0,0 +1,38 @@
+/**
+ * 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 org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The asynchronous connection for internal usage.
+ */
+@InterfaceAudience.Private
+public interface AsyncClusterConnection extends AsyncConnection {
+
+  /**
+   * Get the nonce generator for this connection.
+   */
+  NonceGenerator getNonceGenerator();
+
+  /**
+   * Get the rpc client we used to communicate with other servers.
+   */
+  RpcClient getRpcClient();
+}
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 e47d4cc..f8ad97a 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
@@ -23,6 +23,7 @@ import static org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLE
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import java.io.IOException;
+import java.net.SocketAddress;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -63,7 +64,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterServ
  * The implementation of AsyncConnection.
  */
 @InterfaceAudience.Private
-class AsyncConnectionImpl implements AsyncConnection {
+class AsyncConnectionImpl implements AsyncClusterConnection {
 
   private static final Logger LOG = LoggerFactory.getLogger(AsyncConnectionImpl.class);
 
@@ -108,7 +109,7 @@ class AsyncConnectionImpl implements AsyncConnection {
   private volatile boolean closed = false;
 
   public AsyncConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId,
-      User user) {
+      SocketAddress localAddress, User user) {
     this.conf = conf;
     this.user = user;
     if (user.isLoginFromKeytab()) {
@@ -116,7 +117,7 @@ class AsyncConnectionImpl implements AsyncConnection {
     }
     this.connConf = new AsyncConnectionConfiguration(conf);
     this.registry = registry;
-    this.rpcClient = RpcClientFactory.createClient(conf, clusterId);
+    this.rpcClient = RpcClientFactory.createClient(conf, clusterId, localAddress, null);
     this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
     this.hostnameCanChange = conf.getBoolean(RESOLVE_HOSTNAME_ON_FAIL_KEY, true);
     this.rpcTimeout =
@@ -171,11 +172,16 @@ class AsyncConnectionImpl implements AsyncConnection {
   }
 
   // ditto
-  @VisibleForTesting
+  @Override
   public NonceGenerator getNonceGenerator() {
     return nonceGenerator;
   }
 
+  @Override
+  public RpcClient getRpcClient() {
+    return rpcClient;
+  }
+
   private ClientService.Interface createRegionServerStub(ServerName serverName) throws IOException {
     return ClientService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
new file mode 100644
index 0000000..68c0630
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
@@ -0,0 +1,63 @@
+/**
+ * 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 java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.SocketAddress;
+import java.util.concurrent.ExecutionException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+
+/**
+ * The factory for creating {@link AsyncClusterConnection}.
+ */
+@InterfaceAudience.Private
+public final class ClusterConnectionFactory {
+
+  private ClusterConnectionFactory() {
+  }
+
+  /**
+   * Create a new {@link AsyncClusterConnection} instance.
+   * <p/>
+   * Unlike what we have done in {@link ConnectionFactory}, here we just return an
+   * {@link AsyncClusterConnection} instead of a {@link java.util.concurrent.CompletableFuture},
+   * which means this method could block on fetching the cluster id. This is just used to simplify
+   * the implementation, as when starting new region servers, we do not need to be event-driven. Can
+   * change later if we want a {@link java.util.concurrent.CompletableFuture} here.
+   */
+  public static AsyncClusterConnection createAsyncClusterConnection(Configuration conf,
+      SocketAddress localAddress, User user) throws IOException {
+    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf);
+    String clusterId;
+    try {
+      clusterId = registry.getClusterId().get();
+    } catch (InterruptedException e) {
+      throw (IOException) new InterruptedIOException().initCause(e);
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      Throwables.propagateIfPossible(cause, IOException.class);
+      throw new IOException(cause);
+    }
+    return new AsyncConnectionImpl(conf, registry, clusterId, localAddress, user);
+  }
+}
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 b36485f..b984a99 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
@@ -298,7 +298,7 @@ public class ConnectionFactory {
       try {
         future.complete(
           user.runAs((PrivilegedExceptionAction<? extends AsyncConnection>) () -> ReflectionUtils
-            .newInstance(clazz, conf, registry, clusterId, user)));
+            .newInstance(clazz, conf, registry, clusterId, null, user)));
       } catch (Exception e) {
         registry.close();
         future.completeExceptionally(e);
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 46d786e..d4db7a4 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",
+    conn = new AsyncConnectionImpl(CONF, new DoNothingAsyncRegistry(CONF), "test", null,
       UserProvider.instantiate(CONF).getCurrent()) {
 
       @Override
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 56dcf10..15429cb 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",
+    conn = new AsyncConnectionImpl(CONF, new DoNothingAsyncRegistry(CONF), "test", null,
       UserProvider.instantiate(CONF).getCurrent()) {
 
       @Override
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
index a136846..268249d 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
@@ -83,15 +83,19 @@ public class ReflectionUtils {
 
       boolean match = true;
       for (int i = 0; i < ctorParamTypes.length && match; ++i) {
-        Class<?> paramType = paramTypes[i].getClass();
-        match = (!ctorParamTypes[i].isPrimitive()) ? ctorParamTypes[i].isAssignableFrom(paramType) :
-                  ((int.class.equals(ctorParamTypes[i]) && Integer.class.equals(paramType)) ||
-                   (long.class.equals(ctorParamTypes[i]) && Long.class.equals(paramType)) ||
-                   (double.class.equals(ctorParamTypes[i]) && Double.class.equals(paramType)) ||
-                   (char.class.equals(ctorParamTypes[i]) && Character.class.equals(paramType)) ||
-                   (short.class.equals(ctorParamTypes[i]) && Short.class.equals(paramType)) ||
-                   (boolean.class.equals(ctorParamTypes[i]) && Boolean.class.equals(paramType)) ||
-                   (byte.class.equals(ctorParamTypes[i]) && Byte.class.equals(paramType)));
+        if (paramTypes[i] == null) {
+          match = !ctorParamTypes[i].isPrimitive();
+        } else {
+          Class<?> paramType = paramTypes[i].getClass();
+          match = (!ctorParamTypes[i].isPrimitive()) ? ctorParamTypes[i].isAssignableFrom(paramType)
+            : ((int.class.equals(ctorParamTypes[i]) && Integer.class.equals(paramType)) ||
+              (long.class.equals(ctorParamTypes[i]) && Long.class.equals(paramType)) ||
+              (double.class.equals(ctorParamTypes[i]) && Double.class.equals(paramType)) ||
+              (char.class.equals(ctorParamTypes[i]) && Character.class.equals(paramType)) ||
+              (short.class.equals(ctorParamTypes[i]) && Short.class.equals(paramType)) ||
+              (boolean.class.equals(ctorParamTypes[i]) && Boolean.class.equals(paramType)) ||
+              (byte.class.equals(ctorParamTypes[i]) && Byte.class.equals(paramType)));
+        }
       }
 
       if (match) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
index fb898ea..c33d5af 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -61,6 +63,24 @@ public interface Server extends Abortable, Stoppable {
   ClusterConnection getClusterConnection();
 
   /**
+   * Returns a reference to the servers' async connection.
+   * <p/>
+   * Important note: this method returns a reference to Connection which is managed by Server
+   * itself, so callers must NOT attempt to close connection obtained.
+   */
+  default AsyncConnection getAsyncConnection() {
+    return getAsyncClusterConnection();
+  }
+
+  /**
+   * Returns a reference to the servers' async cluster connection.
+   * <p/>
+   * Important note: this method returns a reference to Connection which is managed by Server
+   * itself, so callers must NOT attempt to close connection obtained.
+   */
+  AsyncClusterConnection getAsyncClusterConnection();
+
+  /**
    * @return The unique server name for this server.
    */
   ServerName getServerName();
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 10bfade..8aa63fe 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
@@ -3039,6 +3039,9 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (this.clusterConnection != null) {
       this.clusterConnection.close();
     }
+    if (this.asyncClusterConnection != null) {
+      this.asyncClusterConnection.close();
+    }
   }
 
   public void stopMaster() throws IOException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index f983882..ba25e81 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -82,7 +82,9 @@ import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.ZNodeClearer;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -109,7 +111,6 @@ import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
 import org.apache.hadoop.hbase.ipc.NettyRpcClientConfigHelper;
 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.ipc.RpcServer;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
@@ -268,6 +269,11 @@ public class HRegionServer extends HasThread implements
   protected ClusterConnection clusterConnection;
 
   /**
+   * The asynchronous cluster connection to be shared by services.
+   */
+  protected AsyncClusterConnection asyncClusterConnection;
+
+  /**
    * Go here to get table descriptors.
    */
   protected TableDescriptors tableDescriptors;
@@ -784,11 +790,7 @@ public class HRegionServer extends HasThread implements
     return true;
   }
 
-  /**
-   * Create a 'smarter' Connection, one that is capable of by-passing RPC if the request is to the
-   * local server; i.e. a short-circuit Connection. Safe to use going to local or remote server.
-   */
-  private ClusterConnection createClusterConnection() throws IOException {
+  private Configuration unsetClientZookeeperQuorum() {
     Configuration conf = this.conf;
     if (conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM) != null) {
       // Use server ZK cluster for server-issued connections, so we clone
@@ -796,11 +798,20 @@ public class HRegionServer extends HasThread implements
       conf = new Configuration(this.conf);
       conf.unset(HConstants.CLIENT_ZOOKEEPER_QUORUM);
     }
+    return conf;
+  }
+
+  /**
+   * Create a 'smarter' Connection, one that is capable of by-passing RPC if the request is to the
+   * local server; i.e. a short-circuit Connection. Safe to use going to local or remote server.
+   */
+  private ClusterConnection createClusterConnection() throws IOException {
     // Create a cluster connection that when appropriate, can short-circuit and go directly to the
     // local server if the request is to the local server bypassing RPC. Can be used for both local
     // and remote invocations.
-    ClusterConnection conn = ConnectionUtils.createShortCircuitConnection(conf, null,
-      userProvider.getCurrent(), serverName, rpcServices, rpcServices);
+    ClusterConnection conn =
+      ConnectionUtils.createShortCircuitConnection(unsetClientZookeeperQuorum(), null,
+        userProvider.getCurrent(), serverName, rpcServices, rpcServices);
     // This is used to initialize the batch thread pool inside the connection implementation.
     // When deploy a fresh cluster, we may first use the cluster connection in InitMetaProcedure,
     // which will be executed inside the PEWorker, and then the batch thread pool will inherit the
@@ -834,9 +845,12 @@ public class HRegionServer extends HasThread implements
   /**
    * Setup our cluster connection if not already initialized.
    */
-  protected synchronized void setupClusterConnection() throws IOException {
+  protected final synchronized void setupClusterConnection() throws IOException {
     if (clusterConnection == null) {
       clusterConnection = createClusterConnection();
+      asyncClusterConnection =
+        ClusterConnectionFactory.createAsyncClusterConnection(unsetClientZookeeperQuorum(),
+          new InetSocketAddress(this.rpcServices.isa.getAddress(), 0), userProvider.getCurrent());
     }
   }
 
@@ -850,8 +864,7 @@ public class HRegionServer extends HasThread implements
       initializeZooKeeper();
       setupClusterConnection();
       // Setup RPC client for master communication
-      this.rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(
-          this.rpcServices.isa.getAddress(), 0), clusterConnection.getConnectionMetrics());
+      this.rpcClient = asyncClusterConnection.getRpcClient();
     } catch (Throwable t) {
       // Call stop if error or process will stick around for ever since server
       // puts up non-daemon threads.
@@ -1115,7 +1128,15 @@ public class HRegionServer extends HasThread implements
         LOG.warn("Attempt to close server's short circuit ClusterConnection failed.", e);
       }
     }
-
+    if (this.asyncClusterConnection != null) {
+      try {
+        this.asyncClusterConnection.close();
+      } catch (IOException e) {
+        // Although the {@link Closeable} interface throws an {@link
+        // IOException}, in reality, the implementation would never do that.
+        LOG.warn("Attempt to close server's AsyncClusterConnection failed.", e);
+      }
+    }
     // Closing the compactSplit thread before closing meta regions
     if (!this.killed && containsMetaTableRegions()) {
       if (!abortRequested || this.fsOk) {
@@ -3754,9 +3775,9 @@ public class HRegionServer extends HasThread implements
   }
 
   @Override
-  public EntityLock regionLock(List<RegionInfo> regionInfos, String description,
-      Abortable abort) throws IOException {
-    return new LockServiceClient(conf, lockStub, clusterConnection.getNonceGenerator())
+  public EntityLock regionLock(List<RegionInfo> regionInfos, String description, Abortable abort)
+      throws IOException {
+    return new LockServiceClient(conf, lockStub, asyncClusterConnection.getNonceGenerator())
       .regionLock(regionInfos, description, abort);
   }
 
@@ -3863,4 +3884,9 @@ public class HRegionServer extends HasThread implements
       Runtime.getRuntime().halt(1);
     }
   }
+
+  @Override
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return asyncClusterConnection;
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index c7bccb3..7d1245c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -180,5 +181,10 @@ public class ReplicationSyncUp extends Configured implements Tool {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index 0e4f241..5205960 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -368,4 +369,8 @@ public class MockRegionServerServices implements RegionServerServices {
   public Optional<MobFileCache> getMobFileCache() {
     return Optional.empty();
   }
+
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return null;
+  }
 }
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 cb09b79..6cdb9de 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
@@ -81,7 +81,7 @@ public class TestAsyncNonMetaRegionLocator {
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-      registry.getClusterId().get(), User.getCurrent());
+      registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = new AsyncNonMetaRegionLocator(CONN);
     SPLIT_KEYS = new byte[8][];
     for (int i = 111; i < 999; i += 111) {
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 8cdb4a9..7e06218 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
@@ -125,7 +125,7 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-      registry.getClusterId().get(), User.getCurrent());
+      registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = new AsyncNonMetaRegionLocator(CONN);
     SPLIT_KEYS = IntStream.range(1, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
       .toArray(byte[][]::new);
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 0a94def..a7cf322 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
@@ -100,7 +100,7 @@ public class TestAsyncRegionLocator {
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-      registry.getClusterId().get(), User.getCurrent());
+      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 7d8956b..29dcd56 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
@@ -73,7 +73,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-      registry.getClusterId().get(), User.getCurrent());
+      registry.getClusterId().get(), null, User.getCurrent());
   }
 
   @AfterClass
@@ -164,7 +164,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
         }
       };
     try (AsyncConnectionImpl mockedConn = new AsyncConnectionImpl(CONN.getConfiguration(),
-      CONN.registry, CONN.registry.getClusterId().get(), User.getCurrent()) {
+      CONN.registry, CONN.registry.getClusterId().get(), null, User.getCurrent()) {
 
       @Override
       AsyncRegionLocator getLocator() {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 9c55f57..3ebad66 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Connection;
@@ -473,4 +474,9 @@ public class MockNoopMasterServices implements MasterServices {
   public SyncReplicationReplayWALManager getSyncReplicationReplayWALManager() {
     return null;
   }
+
+  @Override
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return null;
+  }
 }
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index a930d7f..73d53c7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -721,4 +722,8 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
   public Optional<MobFileCache> getMobFileCache() {
     return Optional.empty();
   }
+
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return null;
+  }
 }
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 2300f54..77667a7 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
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@@ -349,5 +350,10 @@ public class TestActiveMasterManager {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 5c8db3e..c5fad32 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -279,6 +280,11 @@ public class TestHFileCleaner {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   @Test
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
index 119194b..fd11ff8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.HFileLink;
@@ -213,5 +214,10 @@ public class TestHFileLinkCleaner {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 4d254a0..6a5fe9c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -411,6 +412,11 @@ public class TestLogsCleaner {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   static class FaultyZooKeeperWatcher extends ZKWatcher {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index d162bf3..9791643 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -303,6 +304,11 @@ public class TestReplicationHFileCleaner {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   static class FaultyZooKeeperWatcher extends ZKWatcher {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
index 8c9ce75..4a359e4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
@@ -862,6 +863,11 @@ public class TestHeapMemoryManager {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   static class CustomHeapMemoryTuner implements HeapMemoryTuner {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
index 14dc619..43da846 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.SplitLogCounters;
 import org.apache.hadoop.hbase.SplitLogTask;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
@@ -160,6 +161,11 @@ public class TestSplitLogWorker {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   private void waitForCounter(LongAdder ctr, long oldval, long newval, long timems)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 0e20252..9e9d1d6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Durability;
@@ -523,6 +524,11 @@ public class TestWALLockup {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   static class DummyWALActionsListener implements WALActionsListener {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index 863d558..62ab265 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -263,5 +264,10 @@ public class TestReplicationTrackerZKImpl {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 86bbb09..427f319 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
@@ -906,5 +907,10 @@ public abstract class TestReplicationSourceManager {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
index e4780f1..92c8e54 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -363,6 +364,11 @@ public class TestTokenAuthentication {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   @Parameters(name = "{index}: rpcServerImpl={0}")
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
index c25db01..13212d2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
@@ -143,4 +144,9 @@ public class MockServer implements Server {
   public Connection createConnection(Configuration conf) throws IOException {
     return null;
   }
+
+  @Override
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return null;
+  }
 }


[hbase] 05/11: HBASE-21538 Rewrite RegionReplicaFlushHandler to use AsyncClusterConnection

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

zhangduo pushed a commit to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 6220997a664580bb0157f2c1e0eebf922a107ae4
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Wed Dec 12 09:33:33 2018 +0800

    HBASE-21538 Rewrite RegionReplicaFlushHandler to use AsyncClusterConnection
---
 .../hbase/client/AsyncClusterConnection.java       |   8 ++
 .../hadoop/hbase/client/AsyncConnectionImpl.java   |   8 ++
 .../hbase/client/ClusterConnectionFactory.java     |  16 +--
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |  36 ++++---
 .../hbase/protobuf/ReplicationProtbufUtil.java     |  15 +--
 .../hadoop/hbase/regionserver/HRegionServer.java   |   3 +-
 .../handler/RegionReplicaFlushHandler.java         | 110 ++++++++++++---------
 7 files changed, 106 insertions(+), 90 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
index 1327fd7..f1f64ca 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
@@ -17,10 +17,13 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.util.concurrent.CompletableFuture;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
+
 /**
  * The asynchronous connection for internal usage.
  */
@@ -41,4 +44,9 @@ public interface AsyncClusterConnection extends AsyncConnection {
    * Get the rpc client we used to communicate with other servers.
    */
   RpcClient getRpcClient();
+
+  /**
+   * Flush a region and get the response.
+   */
+  CompletableFuture<FlushRegionResponse> flush(byte[] regionName, boolean writeFlushWALMarker);
 }
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 baeb1ef..8386ffa 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
@@ -55,6 +55,7 @@ import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
 
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
@@ -382,4 +383,11 @@ class AsyncConnectionImpl implements AsyncClusterConnection {
   public AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName) {
     return new AsyncRegionServerAdmin(serverName, this);
   }
+
+  @Override
+  public CompletableFuture<FlushRegionResponse> flush(byte[] regionName,
+      boolean writeFlushWALMarker) {
+    RawAsyncHBaseAdmin admin = (RawAsyncHBaseAdmin) getAdmin();
+    return admin.flushRegionInternal(regionName, writeFlushWALMarker);
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
index 68c0630..79484db 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
@@ -18,15 +18,12 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
-import java.io.InterruptedIOException;
 import java.net.SocketAddress;
-import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
-
 /**
  * The factory for creating {@link AsyncClusterConnection}.
  */
@@ -48,16 +45,7 @@ public final class ClusterConnectionFactory {
   public static AsyncClusterConnection createAsyncClusterConnection(Configuration conf,
       SocketAddress localAddress, User user) throws IOException {
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf);
-    String clusterId;
-    try {
-      clusterId = registry.getClusterId().get();
-    } catch (InterruptedException e) {
-      throw (IOException) new InterruptedIOException().initCause(e);
-    } catch (ExecutionException e) {
-      Throwable cause = e.getCause();
-      Throwables.propagateIfPossible(cause, IOException.class);
-      throw new IOException(cause);
-    }
+    String clusterId = FutureUtils.get(registry.getClusterId());
     return new AsyncConnectionImpl(conf, registry, clusterId, localAddress, user);
   }
 }
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 04ed3c5..3021e73 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
@@ -870,7 +870,19 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   @Override
   public CompletableFuture<Void> flushRegion(byte[] regionName) {
-    CompletableFuture<Void> future = new CompletableFuture<>();
+    return flushRegionInternal(regionName, false).thenAccept(r -> {
+    });
+  }
+
+  /**
+   * This method is for internal use only, where we need the response of the flush.
+   * <p/>
+   * As it exposes the protobuf message, please do <strong>NOT</strong> try to expose it as a public
+   * API.
+   */
+  CompletableFuture<FlushRegionResponse> flushRegionInternal(byte[] regionName,
+      boolean writeFlushWALMarker) {
+    CompletableFuture<FlushRegionResponse> future = new CompletableFuture<>();
     addListener(getRegionLocation(regionName), (location, err) -> {
       if (err != null) {
         future.completeExceptionally(err);
@@ -882,7 +894,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
           .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName)));
         return;
       }
-      addListener(flush(serverName, location.getRegion()), (ret, err2) -> {
+      addListener(flush(serverName, location.getRegion(), writeFlushWALMarker), (ret, err2) -> {
         if (err2 != null) {
           future.completeExceptionally(err2);
         } else {
@@ -893,15 +905,14 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     return future;
   }
 
-  private CompletableFuture<Void> flush(final ServerName serverName, final RegionInfo regionInfo) {
-    return this.<Void> newAdminCaller()
-            .serverName(serverName)
-            .action(
-              (controller, stub) -> this.<FlushRegionRequest, FlushRegionResponse, Void> adminCall(
-                controller, stub, RequestConverter.buildFlushRegionRequest(regionInfo
-                  .getRegionName()), (s, c, req, done) -> s.flushRegion(c, req, done),
-                resp -> null))
-            .call();
+  private CompletableFuture<FlushRegionResponse> flush(ServerName serverName, RegionInfo regionInfo,
+      boolean writeFlushWALMarker) {
+    return this.<FlushRegionResponse> newAdminCaller().serverName(serverName)
+      .action((controller, stub) -> this
+        .<FlushRegionRequest, FlushRegionResponse, FlushRegionResponse> adminCall(controller, stub,
+          RequestConverter.buildFlushRegionRequest(regionInfo.getRegionName(), writeFlushWALMarker),
+          (s, c, req, done) -> s.flushRegion(c, req, done), resp -> resp))
+      .call();
   }
 
   @Override
@@ -914,7 +925,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       }
       List<CompletableFuture<Void>> compactFutures = new ArrayList<>();
       if (hRegionInfos != null) {
-        hRegionInfos.forEach(region -> compactFutures.add(flush(sn, region)));
+        hRegionInfos.forEach(region -> compactFutures.add(flush(sn, region, false).thenAccept(r -> {
+        })));
       }
       addListener(CompletableFuture.allOf(
         compactFutures.toArray(new CompletableFuture<?>[compactFutures.size()])), (ret, err2) -> {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
index 74fad26..9f41a76 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
@@ -18,13 +18,10 @@
  */
 package org.apache.hadoop.hbase.protobuf;
 
-
 import java.io.IOException;
-import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
@@ -32,12 +29,12 @@ import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.io.SizedCellScanner;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
@@ -60,15 +57,7 @@ public class ReplicationProtbufUtil {
       throws IOException {
     Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p = buildReplicateWALEntryRequest(
       entries, null, replicationClusterId, sourceBaseNamespaceDir, sourceHFileArchiveDir);
-    try {
-      admin.replicateWALEntry(p.getFirst(), p.getSecond()).get();
-    } catch (InterruptedException e) {
-      throw (IOException) new InterruptedIOException().initCause(e);
-    } catch (ExecutionException e) {
-      Throwable cause = e.getCause();
-      Throwables.propagateIfPossible(cause, IOException.class);
-      throw new IOException(e);
-    }
+    FutureUtils.get(admin.replicateWALEntry(p.getFirst(), p.getSecond()));
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 73ff37a..710efc1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -2395,8 +2395,7 @@ public class HRegionServer extends HasThread implements
 
     // submit it to be handled by one of the handlers so that we do not block OpenRegionHandler
     if (this.executorService != null) {
-      this.executorService.submit(new RegionReplicaFlushHandler(this, clusterConnection,
-          rpcRetryingCallerFactory, rpcControllerFactory, operationTimeout, region));
+      this.executorService.submit(new RegionReplicaFlushHandler(this, region));
     }
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RegionReplicaFlushHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RegionReplicaFlushHandler.java
index 81b6d7e..0729203 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RegionReplicaFlushHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RegionReplicaFlushHandler.java
@@ -20,26 +20,23 @@ package org.apache.hadoop.hbase.regionserver.handler;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.FlushRegionCallable;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.RetryCounter;
 import org.apache.hadoop.hbase.util.RetryCounterFactory;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
 
 /**
  * HBASE-11580: With the async wal approach (HBASE-11568), the edits are not persisted to wal in
@@ -56,20 +53,13 @@ public class RegionReplicaFlushHandler extends EventHandler {
 
   private static final Logger LOG = LoggerFactory.getLogger(RegionReplicaFlushHandler.class);
 
-  private final ClusterConnection connection;
-  private final RpcRetryingCallerFactory rpcRetryingCallerFactory;
-  private final RpcControllerFactory rpcControllerFactory;
-  private final int operationTimeout;
+  private final AsyncClusterConnection connection;
+
   private final HRegion region;
 
-  public RegionReplicaFlushHandler(Server server, ClusterConnection connection,
-      RpcRetryingCallerFactory rpcRetryingCallerFactory, RpcControllerFactory rpcControllerFactory,
-      int operationTimeout, HRegion region) {
+  public RegionReplicaFlushHandler(Server server, HRegion region) {
     super(server, EventType.RS_REGION_REPLICA_FLUSH);
-    this.connection = connection;
-    this.rpcRetryingCallerFactory = rpcRetryingCallerFactory;
-    this.rpcControllerFactory = rpcControllerFactory;
-    this.operationTimeout = operationTimeout;
+    this.connection = server.getAsyncClusterConnection();
     this.region = region;
   }
 
@@ -103,7 +93,7 @@ public class RegionReplicaFlushHandler extends EventHandler {
     return numRetries;
   }
 
-  void triggerFlushInPrimaryRegion(final HRegion region) throws IOException, RuntimeException {
+  void triggerFlushInPrimaryRegion(final HRegion region) throws IOException {
     long pause = connection.getConfiguration().getLong(HConstants.HBASE_CLIENT_PAUSE,
       HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
 
@@ -117,45 +107,59 @@ public class RegionReplicaFlushHandler extends EventHandler {
     }
     while (!region.isClosing() && !region.isClosed()
         && !server.isAborted() && !server.isStopped()) {
-      FlushRegionCallable flushCallable = new FlushRegionCallable(
-        connection, rpcControllerFactory,
-        RegionReplicaUtil.getRegionInfoForDefaultReplica(region.getRegionInfo()), true);
-
       // TODO: flushRegion() is a blocking call waiting for the flush to complete. Ideally we
       // do not have to wait for the whole flush here, just initiate it.
-      FlushRegionResponse response = null;
+      FlushRegionResponse response;
       try {
-         response = rpcRetryingCallerFactory.<FlushRegionResponse>newCaller()
-          .callWithRetries(flushCallable, this.operationTimeout);
-      } catch (IOException ex) {
-        if (ex instanceof TableNotFoundException
-            || connection.isTableDisabled(region.getRegionInfo().getTable())) {
+        response = FutureUtils.get(connection.flush(ServerRegionReplicaUtil
+          .getRegionInfoForDefaultReplica(region.getRegionInfo()).getRegionName(), true));
+      } catch (IOException e) {
+        if (e instanceof TableNotFoundException || FutureUtils
+          .get(connection.getAdmin().isTableDisabled(region.getRegionInfo().getTable()))) {
           return;
         }
-        throw ex;
+        if (!counter.shouldRetry()) {
+          throw e;
+        }
+        // The reason that why we need to retry here is that, the retry for asynchronous admin
+        // request is much simpler than the normal operation, if we failed to locate the region once
+        // then we will throw the exception out and will not try to relocate again. So here we need
+        // to add some retries by ourselves to prevent shutting down the region server too
+        // frequent...
+        LOG.debug("Failed to trigger a flush of primary region replica {} of region {}, retry={}",
+          ServerRegionReplicaUtil.getRegionInfoForDefaultReplica(region.getRegionInfo())
+            .getRegionNameAsString(),
+          region.getRegionInfo().getRegionNameAsString(), counter.getAttemptTimes(), e);
+        try {
+          counter.sleepUntilNextRetry();
+        } catch (InterruptedException e1) {
+          throw new InterruptedIOException(e1.getMessage());
+        }
+        continue;
       }
 
       if (response.getFlushed()) {
         // then we have to wait for seeing the flush entry. All reads will be rejected until we see
         // a complete flush cycle or replay a region open event
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Successfully triggered a flush of primary region replica "
-              + ServerRegionReplicaUtil
-                .getRegionInfoForDefaultReplica(region.getRegionInfo()).getEncodedName()
-                + " of region " + region.getRegionInfo().getEncodedName()
-                + " Now waiting and blocking reads until observing a full flush cycle");
+          LOG.debug("Successfully triggered a flush of primary region replica " +
+            ServerRegionReplicaUtil.getRegionInfoForDefaultReplica(region.getRegionInfo())
+              .getRegionNameAsString() +
+            " of region " + region.getRegionInfo().getRegionNameAsString() +
+            " Now waiting and blocking reads until observing a full flush cycle");
         }
         region.setReadsEnabled(true);
         break;
       } else {
         if (response.hasWroteFlushWalMarker()) {
-          if(response.getWroteFlushWalMarker()) {
+          if (response.getWroteFlushWalMarker()) {
             if (LOG.isDebugEnabled()) {
-              LOG.debug("Successfully triggered an empty flush marker(memstore empty) of primary "
-                  + "region replica " + ServerRegionReplicaUtil
-                    .getRegionInfoForDefaultReplica(region.getRegionInfo()).getEncodedName()
-                  + " of region " + region.getRegionInfo().getEncodedName() + " Now waiting and "
-                  + "blocking reads until observing a flush marker");
+              LOG.debug("Successfully triggered an empty flush marker(memstore empty) of primary " +
+                "region replica " +
+                ServerRegionReplicaUtil.getRegionInfoForDefaultReplica(region.getRegionInfo())
+                  .getRegionNameAsString() +
+                " of region " + region.getRegionInfo().getRegionNameAsString() +
+                " Now waiting and " + "blocking reads until observing a flush marker");
             }
             region.setReadsEnabled(true);
             break;
@@ -164,15 +168,23 @@ public class RegionReplicaFlushHandler extends EventHandler {
             // closing or already flushing. Retry flush again after some sleep.
             if (!counter.shouldRetry()) {
               throw new IOException("Cannot cause primary to flush or drop a wal marker after " +
-                  "retries. Failing opening of this region replica "
-                  + region.getRegionInfo().getEncodedName());
+                counter.getAttemptTimes() + " retries. Failing opening of this region replica " +
+                region.getRegionInfo().getRegionNameAsString());
+            } else {
+              LOG.warn(
+                "Cannot cause primary replica {} to flush or drop a wal marker " +
+                  "for region replica {}, retry={}",
+                ServerRegionReplicaUtil.getRegionInfoForDefaultReplica(region.getRegionInfo())
+                  .getRegionNameAsString(),
+                region.getRegionInfo().getRegionNameAsString(), counter.getAttemptTimes());
             }
           }
         } else {
           // nothing to do. Are we dealing with an old server?
-          LOG.warn("Was not able to trigger a flush from primary region due to old server version? "
-              + "Continuing to open the secondary region replica: "
-              + region.getRegionInfo().getEncodedName());
+          LOG.warn(
+            "Was not able to trigger a flush from primary region due to old server version? " +
+              "Continuing to open the secondary region replica: " +
+              region.getRegionInfo().getRegionNameAsString());
           region.setReadsEnabled(true);
           break;
         }


[hbase] 11/11: HBASE-21585 Remove ClusterConnection

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

zhangduo pushed a commit to branch HBASE-21512
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit e4bded6ca6b6e59504fd08a3440f5f02f12d4f5d
Author: zhangduo <zh...@apache.org>
AuthorDate: Mon Feb 11 20:32:21 2019 +0800

    HBASE-21585 Remove ClusterConnection
---
 .../hadoop/hbase/client/BufferedMutatorImpl.java   |   5 +-
 .../client/CancellableRegionServerCallable.java    |   6 +-
 .../hadoop/hbase/client/ClientServiceCallable.java |  12 +-
 .../hadoop/hbase/client/ClusterConnection.java     | 183 ---------------------
 .../hadoop/hbase/client/ConnectionFactory.java     |   2 +-
 .../hbase/client/ConnectionImplementation.java     | 140 ++++++++++++----
 .../hadoop/hbase/client/ConnectionUtils.java       |  12 +-
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java |  64 +++----
 .../org/apache/hadoop/hbase/client/HBaseHbck.java  |  17 +-
 .../org/apache/hadoop/hbase/client/HTable.java     |  27 ++-
 .../java/org/apache/hadoop/hbase/client/Hbck.java  |   4 +-
 .../apache/hadoop/hbase/client/MasterCallable.java |   7 +-
 .../hadoop/hbase/client/MultiServerCallable.java   |  12 +-
 .../hbase/client/NoncedRegionServerCallable.java   |   4 +-
 .../hbase/client/RegionCoprocessorRpcChannel.java  |   4 +-
 .../hadoop/hbase/client/RegionServerCallable.java  |  12 +-
 .../hadoop/hbase/client/ReversedClientScanner.java |   7 -
 .../hbase/client/ReversedScannerCallable.java      |   8 +-
 .../hadoop/hbase/client/ScannerCallable.java       |  11 +-
 .../hbase/client/ScannerCallableWithReplicas.java  |   2 +-
 .../hadoop/hbase/client/TestAsyncProcess.java      |   7 +-
 .../hadoop/hbase/client/TestBufferedMutator.java   |   3 +-
 .../hadoop/hbase/DistributedHBaseCluster.java      |  27 +--
 .../mapreduce/TestMultiTableInputFormatBase.java   |   4 +-
 .../hbase/mapreduce/TestTableInputFormatBase.java  |  10 +-
 .../main/java/org/apache/hadoop/hbase/Server.java  |   9 -
 .../hbase/backup/example/ZKTableArchiveClient.java |  11 +-
 .../hbase/client/AsyncClusterConnection.java       |   2 +-
 .../org/apache/hadoop/hbase/master/HMaster.java    |  10 +-
 .../master/assignment/AssignmentManagerUtil.java   |  20 +--
 .../quotas/RegionServerSpaceQuotaManager.java      |   2 +-
 .../DisableTableViolationPolicyEnforcement.java    |  16 +-
 .../hadoop/hbase/regionserver/HRegionServer.java   |  39 ++---
 .../regionserver/DumpReplicationQueues.java        |   4 +-
 .../regionserver/ReplicationSyncUp.java            |   8 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java    |  39 ++---
 .../apache/hadoop/hbase/util/HBaseFsckRepair.java  |   1 -
 .../apache/hadoop/hbase/util/MultiHConnection.java |  15 +-
 .../apache/hadoop/hbase/util/RegionSplitter.java   |  15 +-
 .../main/resources/hbase-webapps/master/table.jsp  |   2 +-
 .../java/org/apache/hadoop/hbase/HBaseCluster.java |  23 ---
 .../org/apache/hadoop/hbase/MiniHBaseCluster.java  |  24 ---
 .../hadoop/hbase/MockRegionServerServices.java     |  10 +-
 .../example/TestZooKeeperTableArchiveClient.java   |   6 +-
 .../hbase/client/HConnectionTestingUtility.java    |  20 ++-
 .../org/apache/hadoop/hbase/client/TestAdmin1.java |  29 ++--
 .../org/apache/hadoop/hbase/client/TestAdmin2.java |   2 +-
 .../hbase/client/TestAsyncTableAdminApi.java       |   2 +-
 .../apache/hadoop/hbase/client/TestCISleep.java    |  12 +-
 .../hadoop/hbase/client/TestClientPushback.java    |   6 +-
 .../hbase/client/TestConnectionImplementation.java |   5 +-
 .../hadoop/hbase/client/TestFromClientSide3.java   |   6 +-
 .../client/TestMetaTableAccessorNoCluster.java     |  10 +-
 .../hadoop/hbase/client/TestMetaWithReplicas.java  |  37 ++---
 .../hadoop/hbase/client/TestMultiParallel.java     |   5 +-
 .../hadoop/hbase/client/TestReplicasClient.java    |  22 ++-
 .../hbase/client/TestSeparateClientZKCluster.java  |   2 +-
 .../hbase/client/TestShortCircuitConnection.java   |   2 +-
 .../hbase/master/MockNoopMasterServices.java       |   8 +-
 .../hadoop/hbase/master/MockRegionServer.java      |   8 +-
 .../hbase/master/TestActiveMasterManager.java      |   9 +-
 .../hbase/master/TestClockSkewDetection.java       |  13 +-
 .../hadoop/hbase/master/TestMasterNoCluster.java   |  13 +-
 .../master/assignment/MockMasterServices.java      |   6 +-
 .../hbase/master/cleaner/TestHFileCleaner.java     |   9 +-
 .../hbase/master/cleaner/TestHFileLinkCleaner.java |  10 +-
 .../hbase/master/cleaner/TestLogsCleaner.java      |   8 +-
 .../cleaner/TestReplicationHFileCleaner.java       |   9 +-
 .../procedure/MasterProcedureTestingUtility.java   |   4 +-
 .../regionserver/TestHRegionServerBulkLoad.java    |  35 +---
 .../hbase/regionserver/TestHeapMemoryManager.java  |   9 +-
 .../hbase/regionserver/TestSplitLogWorker.java     |   9 +-
 .../hadoop/hbase/regionserver/TestWALLockup.java   |   8 +-
 .../replication/TestReplicationTrackerZKImpl.java  |   9 +-
 .../regionserver/TestReplicationSourceManager.java |  10 +-
 .../security/token/TestTokenAuthentication.java    |   9 +-
 .../hadoop/hbase/util/BaseTestHBaseFsck.java       |  13 +-
 .../org/apache/hadoop/hbase/util/MockServer.java   |  10 +-
 .../hadoop/hbase/util/MultiThreadedAction.java     |   6 +-
 .../apache/hadoop/hbase/util/TestHBaseFsckMOB.java |   3 +-
 .../hbase/thrift2/client/ThriftConnection.java     |   2 -
 .../hadoop/hbase/thrift2/TestThriftConnection.java |   4 +-
 82 files changed, 423 insertions(+), 807 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
index 922611b..6d70219 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
@@ -63,7 +63,7 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class BufferedMutatorImpl implements BufferedMutator {
+class BufferedMutatorImpl implements BufferedMutator {
 
   private static final Logger LOG = LoggerFactory.getLogger(BufferedMutatorImpl.class);
 
@@ -95,7 +95,8 @@ public class BufferedMutatorImpl implements BufferedMutator {
   private final AsyncProcess ap;
 
   @VisibleForTesting
-  BufferedMutatorImpl(ClusterConnection conn, BufferedMutatorParams params, AsyncProcess ap) {
+  BufferedMutatorImpl(ConnectionImplementation conn, BufferedMutatorParams params,
+      AsyncProcess ap) {
     if (conn == null || conn.isClosed()) {
       throw new IllegalArgumentException("Connection is null or closed.");
     }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java
index 6ad9254..f81018e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java
@@ -39,8 +39,10 @@ abstract class CancellableRegionServerCallable<T> extends ClientServiceCallable<
     Cancellable {
   private final RetryingTimeTracker tracker;
   private final int rpcTimeout;
-  CancellableRegionServerCallable(Connection connection, TableName tableName, byte[] row,
-      RpcController rpcController, int rpcTimeout, RetryingTimeTracker tracker, int priority) {
+
+  CancellableRegionServerCallable(ConnectionImplementation connection, TableName tableName,
+      byte[] row, RpcController rpcController, int rpcTimeout, RetryingTimeTracker tracker,
+      int priority) {
     super(connection, tableName, row, rpcController, priority);
     this.rpcTimeout = rpcTimeout;
     this.tracker = tracker;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java
index 67ba838..c7006a8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java
@@ -31,12 +31,12 @@ import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
  * @param <T>
  */
 @InterfaceAudience.Private
-public abstract class ClientServiceCallable<T> extends
-    RegionServerCallable<T, ClientProtos.ClientService.BlockingInterface> {
+public abstract class ClientServiceCallable<T>
+    extends RegionServerCallable<T, ClientProtos.ClientService.BlockingInterface> {
 
-  public ClientServiceCallable(Connection connection, TableName tableName, byte[] row,
+  public ClientServiceCallable(ConnectionImplementation connection, TableName tableName, byte[] row,
       RpcController rpcController, int priority) {
-    super((ConnectionImplementation) connection, tableName, row, rpcController, priority);
+    super(connection, tableName, row, rpcController, priority);
   }
 
   @Override
@@ -46,12 +46,12 @@ public abstract class ClientServiceCallable<T> extends
 
   // Below here are simple methods that contain the stub and the rpcController.
   protected ClientProtos.GetResponse doGet(ClientProtos.GetRequest request)
-  throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
+      throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
     return getStub().get(getRpcController(), request);
   }
 
   protected ClientProtos.MutateResponse doMutate(ClientProtos.MutateRequest request)
-  throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
+      throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
     return getStub().mutate(getRpcController(), request);
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
deleted file mode 100644
index 7828ef0..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/**
- *
-
- * 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 java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.MasterNotRunningException;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
-
-/** Internal methods on Connection that should not be used by user code. */
-@InterfaceAudience.Private
-// NOTE: Although this class is public, this class is meant to be used directly from internal
-// classes and unit tests only.
-public interface ClusterConnection extends Connection {
-
-  /**
-   * Key for configuration in Configuration whose value is the class we implement making a
-   * new Connection instance.
-   */
-  String HBASE_CLIENT_CONNECTION_IMPL = "hbase.client.connection.impl";
-
-  /**
-   * @return - true if the master server is running
-   * @deprecated this has been deprecated without a replacement
-   */
-  @Deprecated
-  boolean isMasterRunning()
-      throws MasterNotRunningException, ZooKeeperConnectionException;
-
-  /**
-   * Use this api to check if the table has been created with the specified number of
-   * splitkeys which was used while creating the given table.
-   * Note : If this api is used after a table's region gets splitted, the api may return
-   * false.
-   * @param tableName
-   *          tableName
-   * @param splitKeys
-   *          splitKeys used while creating table
-   * @throws IOException
-   *           if a remote or network exception occurs
-   */
-  boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws
-      IOException;
-
-  /**
-   * A table that isTableEnabled == false and isTableDisabled == false
-   * is possible. This happens when a table has a lot of regions
-   * that must be processed.
-   * @param tableName table name
-   * @return true if the table is enabled, false otherwise
-   * @throws IOException if a remote or network exception occurs
-   */
-  boolean isTableEnabled(TableName tableName) throws IOException;
-
-  /**
-   * @param tableName table name
-   * @return true if the table is disabled, false otherwise
-   * @throws IOException if a remote or network exception occurs
-   */
-  boolean isTableDisabled(TableName tableName) throws IOException;
-
-  /**
-   * Retrieve TableState, represent current table state.
-   * @param tableName table state for
-   * @return state of the table
-   */
-  TableState getTableState(TableName tableName)  throws IOException;
-
-  /**
-   * Returns a {@link MasterKeepAliveConnection} to the active master
-   */
-  MasterKeepAliveConnection getMaster() throws IOException;
-
-  /**
-   * Get the admin service for master.
-   */
-  AdminService.BlockingInterface getAdminForMaster() throws IOException;
-
-  /**
-   * Establishes a connection to the region server at the specified address.
-   * @param serverName the region server to connect to
-   * @return proxy for HRegionServer
-   * @throws IOException if a remote or network exception occurs
-   */
-  AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException;
-
-  /**
-   * Establishes a connection to the region server at the specified address, and returns
-   * a region client protocol.
-   *
-   * @param serverName the region server to connect to
-   * @return ClientProtocol proxy for RegionServer
-   * @throws IOException if a remote or network exception occurs
-   *
-   */
-  ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException;
-
-  /**
-   * @return Nonce generator for this ClusterConnection; may be null if disabled in configuration.
-   */
-  NonceGenerator getNonceGenerator();
-
-  /**
-   * @return Default AsyncProcess associated with this connection.
-   */
-  AsyncProcess getAsyncProcess();
-
-  /**
-   * Returns a new RpcRetryingCallerFactory from the given {@link Configuration}.
-   * This RpcRetryingCallerFactory lets the users create {@link RpcRetryingCaller}s which can be
-   * intercepted with the configured {@link RetryingCallerInterceptor}
-   * @param conf configuration
-   * @return RpcRetryingCallerFactory
-   */
-  RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf);
-
-  /**
-   * @return Connection's RpcRetryingCallerFactory instance
-   */
-  RpcRetryingCallerFactory getRpcRetryingCallerFactory();
-
-  /**
-   * @return Connection's RpcControllerFactory instance
-   */
-  RpcControllerFactory getRpcControllerFactory();
-
-  /**
-   * @return a ConnectionConfiguration object holding parsed configuration values
-   */
-  ConnectionConfiguration getConnectionConfiguration();
-
-  /**
-   * @return the current statistics tracker associated with this connection
-   */
-  ServerStatisticTracker getStatisticsTracker();
-
-  /**
-   * @return the configured client backoff policy
-   */
-  ClientBackoffPolicy getBackoffPolicy();
-
-  /**
-   * @return the MetricsConnection instance associated with this connection.
-   */
-  MetricsConnection getConnectionMetrics();
-
-  /**
-   * @return true when this connection uses a {@link org.apache.hadoop.hbase.codec.Codec} and so
-   *         supports cell blocks.
-   */
-  boolean hasCellBlockSupport();
-
-  /**
-   * @return the number of region servers that are currently running
-   * @throws IOException if a remote or network exception occurs
-   */
-  int getCurrentNrHRS() throws IOException;
-}
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 b984a99..ceef356 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
@@ -212,7 +212,7 @@ public class ConnectionFactory {
    */
   public static Connection createConnection(Configuration conf, ExecutorService pool,
     final User user) throws IOException {
-    String className = conf.get(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL,
+    String className = conf.get(ConnectionUtils.HBASE_CLIENT_CONNECTION_IMPL,
       ConnectionImplementation.class.getName());
     Class<?> clazz;
     try {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index f262020..87ca5e3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -148,7 +148,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Updat
     value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
     justification="Access to the conncurrent hash map is under a lock so should be fine.")
 @InterfaceAudience.Private
-class ConnectionImplementation implements ClusterConnection, Closeable {
+class ConnectionImplementation implements Connection, Closeable {
   public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server";
   private static final Logger LOG = LoggerFactory.getLogger(ConnectionImplementation.class);
 
@@ -350,9 +350,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
    */
   @VisibleForTesting
   static NonceGenerator injectNonceGeneratorForTesting(
-      ClusterConnection conn, NonceGenerator cnm) {
-    ConnectionImplementation connImpl = (ConnectionImplementation)conn;
-    NonceGenerator ng = connImpl.getNonceGenerator();
+      ConnectionImplementation conn, NonceGenerator cnm) {
+    NonceGenerator ng = conn.getNonceGenerator();
     LOG.warn("Nonce generator is being replaced by test code for "
       + cnm.getClass().getName());
     nonceGenerator = cnm;
@@ -452,7 +451,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       }), rpcControllerFactory);
   }
 
-  @Override
+  /**
+   * @return the MetricsConnection instance associated with this connection.
+   */
   public MetricsConnection getConnectionMetrics() {
     return this.metrics;
   }
@@ -596,7 +597,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
    * @deprecated this has been deprecated without a replacement
    */
   @Deprecated
-  @Override
   public boolean isMasterRunning() throws MasterNotRunningException, ZooKeeperConnectionException {
     // When getting the master connection, we check it's running,
     // so if there is no exception, it means we've been able to get a
@@ -624,18 +624,39 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     return reload ? relocateRegion(tableName, row) : locateRegion(tableName, row);
   }
 
-
-  @Override
+  /**
+   * A table that isTableEnabled == false and isTableDisabled == false
+   * is possible. This happens when a table has a lot of regions
+   * that must be processed.
+   * @param tableName table name
+   * @return true if the table is enabled, false otherwise
+   * @throws IOException if a remote or network exception occurs
+   */
   public boolean isTableEnabled(TableName tableName) throws IOException {
     return getTableState(tableName).inStates(TableState.State.ENABLED);
   }
 
-  @Override
+  /**
+   * @param tableName table name
+   * @return true if the table is disabled, false otherwise
+   * @throws IOException if a remote or network exception occurs
+   */
   public boolean isTableDisabled(TableName tableName) throws IOException {
     return getTableState(tableName).inStates(TableState.State.DISABLED);
   }
 
-  @Override
+  /**
+   * Use this api to check if the table has been created with the specified number of
+   * splitkeys which was used while creating the given table.
+   * Note : If this api is used after a table's region gets splitted, the api may return
+   * false.
+   * @param tableName
+   *          tableName
+   * @param splitKeys
+   *          splitKeys used while creating table
+   * @throws IOException
+   *           if a remote or network exception occurs
+   */
   public boolean isTableAvailable(final TableName tableName, @Nullable final byte[][] splitKeys)
       throws IOException {
     checkClosed();
@@ -805,15 +826,14 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   }
 
   /**
-  *
-  * @param tableName table to get regions of
-  * @param row the row
-  * @param useCache Should we use the cache to retrieve the region information.
-  * @param retry do we retry
-  * @param replicaId the replicaId for the region
-  * @return region locations for this row.
-  * @throws IOException if IO failure occurs
-  */
+   * @param tableName table to get regions of
+   * @param row the row
+   * @param useCache Should we use the cache to retrieve the region information.
+   * @param retry do we retry
+   * @param replicaId the replicaId for the region
+   * @return region locations for this row.
+   * @throws IOException if IO failure occurs
+   */
   RegionLocations locateRegion(final TableName tableName, final byte[] row, boolean useCache,
       boolean retry, int replicaId) throws IOException {
     checkClosed();
@@ -1044,6 +1064,10 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     metaCache.clearCache(serverName);
   }
 
+
+  /**
+   * Allows flushing the region cache.
+   */
   @Override
   public void clearRegionLocationCache() {
     metaCache.clearCache();
@@ -1254,12 +1278,19 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     }
   }
 
-  @Override
+  /**
+   * Get the admin service for master.
+   */
   public AdminProtos.AdminService.BlockingInterface getAdminForMaster() throws IOException {
     return getAdmin(get(registry.getMasterAddress()));
   }
 
-  @Override
+  /**
+   * Establishes a connection to the region server at the specified address.
+   * @param serverName the region server to connect to
+   * @return proxy for HRegionServer
+   * @throws IOException if a remote or network exception occurs
+   */
   public AdminProtos.AdminService.BlockingInterface getAdmin(ServerName serverName)
       throws IOException {
     checkClosed();
@@ -1275,7 +1306,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     });
   }
 
-  @Override
+  /**
+   * Establishes a connection to the region server at the specified address, and returns a region
+   * client protocol.
+   * @param serverName the region server to connect to
+   * @return ClientProtocol proxy for RegionServer
+   * @throws IOException if a remote or network exception occurs
+   */
   public BlockingInterface getClient(ServerName serverName) throws IOException {
     checkClosed();
     if (isDeadServer(serverName)) {
@@ -1285,14 +1322,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       serverName, this.hostnamesCanChange);
     return (ClientProtos.ClientService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> {
       BlockingRpcChannel channel =
-          this.rpcClient.createBlockingRpcChannel(serverName, user, rpcTimeout);
+        this.rpcClient.createBlockingRpcChannel(serverName, user, rpcTimeout);
       return ClientProtos.ClientService.newBlockingStub(channel);
     });
   }
 
   final MasterServiceState masterServiceState = new MasterServiceState(this);
 
-  @Override
   public MasterKeepAliveConnection getMaster() throws IOException {
     return getKeepAliveMasterService();
   }
@@ -1904,6 +1940,10 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     cacheLocation(hri.getTable(), source, newHrl);
   }
 
+  /**
+   * Deletes cached locations for the specific region.
+   * @param location The location object for the region, to be purged from cache.
+   */
   void deleteCachedRegionLocation(final HRegionLocation location) {
     metaCache.clearCache(location);
   }
@@ -1982,17 +2022,23 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     metaCache.clearCache(regionInfo);
   }
 
-  @Override
+  /**
+   * @return Default AsyncProcess associated with this connection.
+   */
   public AsyncProcess getAsyncProcess() {
     return asyncProcess;
   }
 
-  @Override
+  /**
+   * @return the current statistics tracker associated with this connection
+   */
   public ServerStatisticTracker getStatisticsTracker() {
     return this.stats;
   }
 
-  @Override
+  /**
+   * @return the configured client backoff policy
+   */
   public ClientBackoffPolicy getBackoffPolicy() {
     return this.backoffPolicy;
   }
@@ -2028,7 +2074,10 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     return this.aborted;
   }
 
-  @Override
+  /**
+   * @return the number of region servers that are currently running
+   * @throws IOException if a remote or network exception occurs
+   */
   public int getCurrentNrHRS() throws IOException {
     return get(this.registry.getCurrentNrHRS());
   }
@@ -2071,12 +2120,18 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     close();
   }
 
-  @Override
+  /**
+   * @return Nonce generator for this ClusterConnection; may be null if disabled in configuration.
+   */
   public NonceGenerator getNonceGenerator() {
     return nonceGenerator;
   }
 
-  @Override
+  /**
+   * Retrieve TableState, represent current table state.
+   * @param tableName table state for
+   * @return state of the table
+   */
   public TableState getTableState(TableName tableName) throws IOException {
     checkClosed();
     TableState tableState = MetaTableAccessor.getTableState(this, tableName);
@@ -2086,28 +2141,43 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     return tableState;
   }
 
-  @Override
+  /**
+   * Returns a new RpcRetryingCallerFactory from the given {@link Configuration}.
+   * This RpcRetryingCallerFactory lets the users create {@link RpcRetryingCaller}s which can be
+   * intercepted with the configured {@link RetryingCallerInterceptor}
+   * @param conf configuration
+   * @return RpcRetryingCallerFactory
+   */
   public RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf) {
     return RpcRetryingCallerFactory
         .instantiate(conf, this.interceptor, this.getStatisticsTracker());
   }
 
-  @Override
+  /**
+   * @return true when this connection uses a {@link org.apache.hadoop.hbase.codec.Codec} and so
+   *         supports cell blocks.
+   */
   public boolean hasCellBlockSupport() {
     return this.rpcClient.hasCellBlockSupport();
   }
 
-  @Override
+  /**
+   * @return a ConnectionConfiguration object holding parsed configuration values
+   */
   public ConnectionConfiguration getConnectionConfiguration() {
     return this.connectionConfig;
   }
 
-  @Override
+  /**
+   * @return Connection's RpcRetryingCallerFactory instance
+   */
   public RpcRetryingCallerFactory getRpcRetryingCallerFactory() {
     return this.rpcCallerFactory;
   }
 
-  @Override
+  /**
+   * @return Connection's RpcControllerFactory instance
+   */
   public RpcControllerFactory getRpcControllerFactory() {
     return this.rpcControllerFactory;
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
index 67a0b09..877ce2b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
@@ -74,6 +74,12 @@ public final class ConnectionUtils {
 
   private static final Logger LOG = LoggerFactory.getLogger(ConnectionUtils.class);
 
+  /**
+   * Key for configuration in Configuration whose value is the class we implement making a new
+   * Connection instance.
+   */
+  public static final String HBASE_CLIENT_CONNECTION_IMPL = "hbase.client.connection.impl";
+
   private ConnectionUtils() {
   }
 
@@ -103,7 +109,7 @@ public final class ConnectionUtils {
    * @param cnm Replaces the nonce generator used, for testing.
    * @return old nonce generator.
    */
-  public static NonceGenerator injectNonceGeneratorForTesting(ClusterConnection conn,
+  public static NonceGenerator injectNonceGeneratorForTesting(ConnectionImplementation conn,
       NonceGenerator cnm) {
     return ConnectionImplementation.injectNonceGeneratorForTesting(conn, cnm);
   }
@@ -180,7 +186,7 @@ public final class ConnectionUtils {
    * @return an short-circuit connection.
    * @throws IOException if IO failure occurred
    */
-  public static ClusterConnection createShortCircuitConnection(final Configuration conf,
+  public static ConnectionImplementation createShortCircuitConnection(final Configuration conf,
       ExecutorService pool, User user, final ServerName serverName,
       final AdminService.BlockingInterface admin, final ClientService.BlockingInterface client)
       throws IOException {
@@ -196,7 +202,7 @@ public final class ConnectionUtils {
    */
   @VisibleForTesting
   public static void setupMasterlessConnection(Configuration conf) {
-    conf.set(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL, MasterlessConnection.class.getName());
+    conf.set(ConnectionUtils.HBASE_CLIENT_CONNECTION_IMPL, MasterlessConnection.class.getName());
   }
 
   /**
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index d16d630..091eee3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -453,7 +453,7 @@ public class HBaseAdmin implements Admin {
 
   /** @return Connection used by this object. */
   @Override
-  public Connection getConnection() {
+  public ConnectionImplementation getConnection() {
     return connection;
   }
 
@@ -545,23 +545,24 @@ public class HBaseAdmin implements Admin {
        operationTimeout, rpcTimeout);
   }
 
-  static TableDescriptor getTableDescriptor(final TableName tableName, Connection connection,
-      RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
-      int operationTimeout, int rpcTimeout) throws IOException {
+  static TableDescriptor getTableDescriptor(final TableName tableName,
+      ConnectionImplementation connection, RpcRetryingCallerFactory rpcCallerFactory,
+      final RpcControllerFactory rpcControllerFactory, int operationTimeout, int rpcTimeout)
+      throws IOException {
     if (tableName == null) return null;
     TableDescriptor td =
-        executeCallable(new MasterCallable<TableDescriptor>(connection, rpcControllerFactory) {
-      @Override
-      protected TableDescriptor rpcCall() throws Exception {
-        GetTableDescriptorsRequest req =
+      executeCallable(new MasterCallable<TableDescriptor>(connection, rpcControllerFactory) {
+        @Override
+        protected TableDescriptor rpcCall() throws Exception {
+          GetTableDescriptorsRequest req =
             RequestConverter.buildGetTableDescriptorsRequest(tableName);
-        GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
-        if (!htds.getTableSchemaList().isEmpty()) {
-          return ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0));
+          GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
+          if (!htds.getTableSchemaList().isEmpty()) {
+            return ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0));
+          }
+          return null;
         }
-        return null;
-      }
-    }, rpcCallerFactory, operationTimeout, rpcTimeout);
+      }, rpcCallerFactory, operationTimeout, rpcTimeout);
     if (td != null) {
       return td;
     }
@@ -574,26 +575,27 @@ public class HBaseAdmin implements Admin {
    *             Connection, RpcRetryingCallerFactory,RpcControllerFactory,int,int)}
    */
   @Deprecated
-  static HTableDescriptor getHTableDescriptor(final TableName tableName, Connection connection,
-      RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
-      int operationTimeout, int rpcTimeout) throws IOException {
+  static HTableDescriptor getHTableDescriptor(final TableName tableName,
+      ConnectionImplementation connection, RpcRetryingCallerFactory rpcCallerFactory,
+      final RpcControllerFactory rpcControllerFactory, int operationTimeout, int rpcTimeout)
+      throws IOException {
     if (tableName == null) {
       return null;
     }
     HTableDescriptor htd =
-        executeCallable(new MasterCallable<HTableDescriptor>(connection, rpcControllerFactory) {
-          @Override
-          protected HTableDescriptor rpcCall() throws Exception {
-            GetTableDescriptorsRequest req =
-                RequestConverter.buildGetTableDescriptorsRequest(tableName);
-            GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
-            if (!htds.getTableSchemaList().isEmpty()) {
-              return new ImmutableHTableDescriptor(
-                  ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0)));
-            }
-            return null;
+      executeCallable(new MasterCallable<HTableDescriptor>(connection, rpcControllerFactory) {
+        @Override
+        protected HTableDescriptor rpcCall() throws Exception {
+          GetTableDescriptorsRequest req =
+            RequestConverter.buildGetTableDescriptorsRequest(tableName);
+          GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
+          if (!htds.getTableSchemaList().isEmpty()) {
+            return new ImmutableHTableDescriptor(
+              ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0)));
           }
-        }, rpcCallerFactory, operationTimeout, rpcTimeout);
+          return null;
+        }
+      }, rpcCallerFactory, operationTimeout, rpcTimeout);
     if (htd != null) {
       return new ImmutableHTableDescriptor(htd);
     }
@@ -2422,8 +2424,8 @@ public class HBaseAdmin implements Admin {
 
     // Check ZK first.
     // If the connection exists, we may have a connection to ZK that does not work anymore
-    try (ClusterConnection connection =
-        (ClusterConnection) ConnectionFactory.createConnection(copyOfConf)) {
+    try (ConnectionImplementation connection =
+      (ConnectionImplementation) ConnectionFactory.createConnection(copyOfConf)) {
       // can throw MasterNotRunningException
       connection.isMasterRunning();
     }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
index a276017..e7d0e59 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
@@ -21,9 +21,14 @@ import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@@ -31,16 +36,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService.BlockingInterface;
 
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
-
-import org.apache.yetus.audience.InterfaceAudience;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 
 /**
- * Use {@link ClusterConnection#getHbck()} to obtain an instance of {@link Hbck} instead of
+ * Use {@link Connection#getHbck()} to obtain an instance of {@link Hbck} instead of
  * constructing an HBaseHbck directly.
  *
  * <p>Connection should be an <i>unmanaged</i> connection obtained via
@@ -55,7 +53,6 @@ import org.slf4j.LoggerFactory;
  * by each thread. Pooling or caching of the instance is not recommended.</p>
  *
  * @see ConnectionFactory
- * @see ClusterConnection
  * @see Hbck
  */
 @InterfaceAudience.Private
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index cc796c8..bae20c5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -155,7 +155,6 @@ public class HTable implements Table {
    * @param rpcControllerFactory The RPC controller factory
    * @param pool ExecutorService to be used.
    */
-  @InterfaceAudience.Private
   protected HTable(final ConnectionImplementation connection,
       final TableBuilderBase builder,
       final RpcRetryingCallerFactory rpcCallerFactory,
@@ -462,22 +461,18 @@ public class HTable implements Table {
   }
 
   public static <R> void doBatchWithCallback(List<? extends Row> actions, Object[] results,
-    Callback<R> callback, ClusterConnection connection, ExecutorService pool, TableName tableName)
-    throws InterruptedIOException, RetriesExhaustedWithDetailsException {
-    int operationTimeout = connection.getConnectionConfiguration().getOperationTimeout();
+      Callback<R> callback, Connection connection, ExecutorService pool, TableName tableName)
+      throws InterruptedIOException, RetriesExhaustedWithDetailsException {
+    ConnectionImplementation connImpl = (ConnectionImplementation) connection;
+    int operationTimeout = connImpl.getConnectionConfiguration().getOperationTimeout();
     int writeTimeout = connection.getConfiguration().getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY,
-        connection.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
-            HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
-    AsyncProcessTask<R> task = AsyncProcessTask.newBuilder(callback)
-            .setPool(pool)
-            .setTableName(tableName)
-            .setRowAccess(actions)
-            .setResults(results)
-            .setOperationTimeout(operationTimeout)
-            .setRpcTimeout(writeTimeout)
-            .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL)
-            .build();
-    AsyncRequestFuture ars = connection.getAsyncProcess().submit(task);
+      connection.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    AsyncProcessTask<R> task =
+      AsyncProcessTask.newBuilder(callback).setPool(pool).setTableName(tableName)
+        .setRowAccess(actions).setResults(results).setOperationTimeout(operationTimeout)
+        .setRpcTimeout(writeTimeout).setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL).build();
+    AsyncRequestFuture ars = connImpl.getAsyncProcess().submit(task);
     ars.waitUntilDone();
     if (ars.hasError()) {
       throw ars.getErrors();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
index e88805c..fdecde9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
-
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -28,7 +27,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 
 /**
- * Hbck fixup tool APIs. Obtain an instance from {@link ClusterConnection#getHbck()} and call
+ * Hbck fixup tool APIs. Obtain an instance from {@link Connection#getHbck()} and call
  * {@link #close()} when done.
  * <p>WARNING: the below methods can damage the cluster. It may leave the cluster in an
  * indeterminate state, e.g. region not assigned, or some hdfs files left behind. After running
@@ -36,7 +35,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
  * procedures to get regions back online. DO AT YOUR OWN RISK. For experienced users only.
  *
  * @see ConnectionFactory
- * @see ClusterConnection
  * @since 2.0.2, 2.1.1
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.HBCK)
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
index 7ae9731..04da2eb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
@@ -43,12 +43,13 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
  */
 @InterfaceAudience.Private
 abstract class MasterCallable<V> implements RetryingCallable<V>, Closeable {
-  protected final ClusterConnection connection;
+  protected final ConnectionImplementation connection;
   protected MasterKeepAliveConnection master;
   private final HBaseRpcController rpcController;
 
-  MasterCallable(final Connection connection, final RpcControllerFactory rpcConnectionFactory) {
-    this.connection = (ClusterConnection) connection;
+  MasterCallable(ConnectionImplementation connection,
+      final RpcControllerFactory rpcConnectionFactory) {
+    this.connection = connection;
     this.rpcController = rpcConnectionFactory.newController();
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
index bfc161e..bf557fa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -31,15 +30,16 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
  * Callable that handles the <code>multi</code> method call going against a single
@@ -52,7 +52,7 @@ class MultiServerCallable extends CancellableRegionServerCallable<MultiResponse>
   private MultiAction multiAction;
   private boolean cellBlock;
 
-  MultiServerCallable(final ClusterConnection connection, final TableName tableName,
+  MultiServerCallable(final ConnectionImplementation connection, final TableName tableName,
       final ServerName location, final MultiAction multi, RpcController rpcController,
       int rpcTimeout, RetryingTimeTracker tracker, int priority) {
     super(connection, tableName, null, rpcController, rpcTimeout, tracker, priority);
@@ -141,7 +141,7 @@ class MultiServerCallable extends CancellableRegionServerCallable<MultiResponse>
   private boolean isCellBlock() {
     // This is not exact -- the configuration could have changed on us after connection was set up
     // but it will do for now.
-    ClusterConnection conn = getConnection();
+    ConnectionImplementation conn = getConnection();
     return conn.hasCellBlockSupport();
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java
index 2da8422..69f4f4a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java
@@ -46,8 +46,8 @@ public abstract class NoncedRegionServerCallable<T> extends ClientServiceCallabl
    * @param tableName Table name to which <code>row</code> belongs.
    * @param row The row we want in <code>tableName</code>.
    */
-  public NoncedRegionServerCallable(Connection connection, TableName tableName, byte [] row,
-      HBaseRpcController rpcController, int priority) {
+  public NoncedRegionServerCallable(ConnectionImplementation connection, TableName tableName,
+      byte[] row, HBaseRpcController rpcController, int priority) {
     super(connection, tableName, row, rpcController, priority);
     this.nonce = getConnection().getNonceGenerator().newNonce();
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java
index 448302c..80371b7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java
@@ -46,7 +46,7 @@ class RegionCoprocessorRpcChannel extends SyncCoprocessorRpcChannel {
   private static final Logger LOG = LoggerFactory.getLogger(RegionCoprocessorRpcChannel.class);
   private final TableName table;
   private final byte [] row;
-  private final ClusterConnection conn;
+  private final ConnectionImplementation conn;
   private byte[] lastRegion;
   private final int operationTimeout;
   private final RpcRetryingCallerFactory rpcCallerFactory;
@@ -57,7 +57,7 @@ class RegionCoprocessorRpcChannel extends SyncCoprocessorRpcChannel {
    * @param table to connect to
    * @param row to locate region with
    */
-  RegionCoprocessorRpcChannel(ClusterConnection conn, TableName table, byte[] row) {
+  RegionCoprocessorRpcChannel(ConnectionImplementation conn, TableName table, byte[] row) {
     this.table = table;
     this.row = row;
     this.conn = conn;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
index 264304e..009544c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -27,11 +26,12 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 
 /**
  * Implementations make a RPC call against a RegionService via a protobuf Service.
@@ -75,12 +75,12 @@ public abstract class RegionServerCallable<T, S> implements RetryingCallable<T>
    * @param tableName Table name to which <code>row</code> belongs.
    * @param row The row we want in <code>tableName</code>.
    */
-  public RegionServerCallable(ConnectionImplementation connection, TableName tableName, byte [] row,
+  public RegionServerCallable(ConnectionImplementation connection, TableName tableName, byte[] row,
       RpcController rpcController) {
     this(connection, tableName, row, rpcController, HConstants.NORMAL_QOS);
   }
 
-  public RegionServerCallable(ConnectionImplementation connection, TableName tableName, byte [] row,
+  public RegionServerCallable(ConnectionImplementation connection, TableName tableName, byte[] row,
       RpcController rpcController, int priority) {
     super();
     this.connection = connection;
@@ -160,7 +160,7 @@ public abstract class RegionServerCallable<T, S> implements RetryingCallable<T>
   }
 
   /**
-   * @return {@link ClusterConnection} instance used by this Callable.
+   * @return {@link ConnectionImplementation} instance used by this Callable.
    */
   protected ConnectionImplementation getConnection() {
     return this.connection;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java
index 34c24c0..2ed037e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java
@@ -37,13 +37,6 @@ public class ReversedClientScanner extends ClientScanner {
   /**
    * Create a new ReversibleClientScanner for the specified table Note that the passed
    * {@link Scan}'s start row maybe changed.
-   * @param conf
-   * @param scan
-   * @param tableName
-   * @param connection
-   * @param pool
-   * @param primaryOperationTimeout
-   * @throws IOException
    */
   public ReversedClientScanner(Configuration conf, Scan scan, TableName tableName,
       ConnectionImplementation connection, RpcRetryingCallerFactory rpcFactory,
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
index 30e541c..6a325b2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
@@ -52,8 +52,8 @@ public class ReversedScannerCallable extends ScannerCallable {
    * @param rpcFactory to create an {@link com.google.protobuf.RpcController} to talk to the
    *          regionserver
    */
-  public ReversedScannerCallable(ClusterConnection connection, TableName tableName, Scan scan,
-      ScanMetrics scanMetrics, RpcControllerFactory rpcFactory) {
+  public ReversedScannerCallable(ConnectionImplementation connection, TableName tableName,
+      Scan scan, ScanMetrics scanMetrics, RpcControllerFactory rpcFactory) {
     super(connection, tableName, scan, scanMetrics, rpcFactory);
   }
 
@@ -66,8 +66,8 @@ public class ReversedScannerCallable extends ScannerCallable {
    *          regionserver
    * @param replicaId the replica id
    */
-  public ReversedScannerCallable(ClusterConnection connection, TableName tableName, Scan scan,
-      ScanMetrics scanMetrics, RpcControllerFactory rpcFactory, int replicaId) {
+  public ReversedScannerCallable(ConnectionImplementation connection, TableName tableName,
+      Scan scan, ScanMetrics scanMetrics, RpcControllerFactory rpcFactory, int replicaId) {
     super(connection, tableName, scan, scanMetrics, rpcFactory, replicaId);
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
index 45b74ef..bf7135f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
@@ -101,23 +101,24 @@ public class ScannerCallable extends ClientServiceCallable<Result[]> {
    * @param scanMetrics the ScanMetrics to used, if it is null, ScannerCallable won't collect
    *          metrics
    * @param rpcControllerFactory factory to use when creating
-   *        {@link com.google.protobuf.RpcController}
+   *          {@link com.google.protobuf.RpcController}
    */
-  public ScannerCallable(ClusterConnection connection, TableName tableName, Scan scan,
+  public ScannerCallable(ConnectionImplementation connection, TableName tableName, Scan scan,
       ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory) {
     this(connection, tableName, scan, scanMetrics, rpcControllerFactory, 0);
   }
+
   /**
-   *
    * @param connection
    * @param tableName
    * @param scan
    * @param scanMetrics
    * @param id the replicaId
    */
-  public ScannerCallable(ClusterConnection connection, TableName tableName, Scan scan,
+  public ScannerCallable(ConnectionImplementation connection, TableName tableName, Scan scan,
       ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory, int id) {
-    super(connection, tableName, scan.getStartRow(), rpcControllerFactory.newController(), scan.getPriority());
+    super(connection, tableName, scan.getStartRow(), rpcControllerFactory.newController(),
+      scan.getPriority());
     this.id = id;
     this.scan = scan;
     this.scanMetrics = scanMetrics;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
index 27e5f87..db956ce 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
@@ -76,7 +76,7 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
   public ScannerCallableWithReplicas(TableName tableName, ConnectionImplementation cConnection,
       ScannerCallable baseCallable, ExecutorService pool, int timeBeforeReplicas, Scan scan,
       int retries, int scannerTimeout, int caching, Configuration conf,
-      RpcRetryingCaller<Result []> caller) {
+      RpcRetryingCaller<Result[]> caller) {
     this.currentScannerCallable = baseCallable;
     this.cConnection = cConnection;
     this.pool = pool;
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index 70d10ca..d4781d1 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -1124,11 +1124,8 @@ public class TestAsyncProcess {
                1,    BufferedMutator.MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS);
   }
 
-  private void checkPeriodicFlushParameters(ClusterConnection conn,
-                                            MyAsyncProcess ap,
-                                            long setTO, long expectTO,
-                                            long setTT, long expectTT
-                                            ) {
+  private void checkPeriodicFlushParameters(ConnectionImplementation conn, MyAsyncProcess ap,
+      long setTO, long expectTO, long setTT, long expectTT) {
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
 
     // The BufferedMutatorParams does nothing with the value
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java
index f0375e2..647ea32 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java
@@ -44,8 +44,7 @@ public class TestBufferedMutator {
   public TestName name = new TestName();
 
   /**
-   * My BufferedMutator.
-   * Just to prove that I can insert a BM other than default.
+   * My BufferedMutator. Just to prove that I can insert a BM other than default.
    */
   public static class MyBufferedMutator extends BufferedMutatorImpl {
     MyBufferedMutator(ConnectionImplementation conn, RpcRetryingCallerFactory rpcCallerFactory,
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
index 30a3db9..2a7b7cd 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Comparator;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Objects;
@@ -28,7 +29,6 @@ import java.util.TreeSet;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterManager.ServiceType;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -37,10 +37,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
-
 /**
  * Manages the interactions with an already deployed distributed cluster (as opposed to
  * a pseudo-distributed, or mini/local cluster). This is used by integration and system tests.
@@ -100,18 +96,6 @@ public class DistributedHBaseCluster extends HBaseCluster {
   }
 
   @Override
-  public AdminProtos.AdminService.BlockingInterface getAdminProtocol(ServerName serverName)
-  throws IOException {
-    return ((ClusterConnection)this.connection).getAdmin(serverName);
-  }
-
-  @Override
-  public ClientProtos.ClientService.BlockingInterface getClientProtocol(ServerName serverName)
-  throws IOException {
-    return ((ClusterConnection)this.connection).getClient(serverName);
-  }
-
-  @Override
   public void startRegionServer(String hostname, int port) throws IOException {
     LOG.info("Starting RS on: " + hostname);
     clusterManager.start(ServiceType.HBASE_REGIONSERVER, hostname, port);
@@ -262,13 +246,6 @@ public class DistributedHBaseCluster extends HBaseCluster {
     throw new IOException("did timeout waiting for service to start:" + serverName);
   }
 
-
-  @Override
-  public MasterService.BlockingInterface getMasterAdminService()
-  throws IOException {
-    return ((ClusterConnection)this.connection).getMaster();
-  }
-
   @Override
   public void startMaster(String hostname, int port) throws IOException {
     LOG.info("Starting Master on: " + hostname + ":" + port);
@@ -297,7 +274,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
     long start = System.currentTimeMillis();
     while (System.currentTimeMillis() - start < timeout) {
       try {
-        getMasterAdminService();
+        connection.getAdmin().getClusterMetrics(EnumSet.of(ClusterMetrics.Option.HBASE_VERSION));
         return true;
       } catch (MasterNotRunningException m) {
         LOG.warn("Master not started yet " + m);
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java
index 906abca..eff26d7 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java
@@ -36,8 +36,8 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.BufferedMutator;
 import org.apache.hadoop.hbase.client.BufferedMutatorParams;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
@@ -98,7 +98,7 @@ public class TestMultiTableInputFormatBase {
     // canned responses.
     JobContext mockedJobContext = Mockito.mock(JobContext.class);
     Configuration c = HBaseConfiguration.create();
-    c.set(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL, MRSplitsConnection.class.getName());
+    c.set(ConnectionUtils.HBASE_CLIENT_CONNECTION_IMPL, MRSplitsConnection.class.getName());
     Mockito.when(mockedJobContext.getConfiguration()).thenReturn(c);
     // Invent a bunch of scans. Have each Scan go against a different table so a good spread.
     List<Scan> scans = new ArrayList<>();
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
index 4436ee1..944bd10 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hbase.mapreduce;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.anyBoolean;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -40,8 +40,8 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.BufferedMutator;
 import org.apache.hadoop.hbase.client.BufferedMutatorParams;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionLocator;
@@ -90,7 +90,7 @@ public class TestTableInputFormatBase {
   public void testNonSuccessiveSplitsAreNotMerged() throws IOException {
     JobContext context = mock(JobContext.class);
     Configuration conf = HBaseConfiguration.create();
-    conf.set(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL,
+    conf.set(ConnectionUtils.HBASE_CLIENT_CONNECTION_IMPL,
         ConnectionForMergeTesting.class.getName());
     conf.set(TableInputFormat.INPUT_TABLE, "testTable");
     conf.setBoolean(TableInputFormatBase.MAPREDUCE_INPUT_AUTOBALANCE, true);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
index c33d5af..e0e95df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.AsyncConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -55,14 +54,6 @@ public interface Server extends Abortable, Stoppable {
   Connection createConnection(Configuration conf) throws IOException;
 
   /**
-   * Returns a reference to the servers' cluster connection. Prefer {@link #getConnection()}.
-   *
-   * Important note: this method returns a reference to Connection which is managed
-   * by Server itself, so callers must NOT attempt to close connection obtained.
-   */
-  ClusterConnection getClusterConnection();
-
-  /**
    * Returns a reference to the servers' async connection.
    * <p/>
    * Important note: this method returns a reference to Connection which is managed by Server
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java
index 142788e..af0d560 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java
@@ -18,14 +18,13 @@
 package org.apache.hadoop.hbase.backup.example;
 
 import java.io.IOException;
-
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.util.Bytes;
+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;
 
 /**
@@ -36,9 +35,9 @@ public class ZKTableArchiveClient extends Configured {
 
   /** Configuration key for the archive node. */
   private static final String ZOOKEEPER_ZNODE_HFILE_ARCHIVE_KEY = "zookeeper.znode.hfile.archive";
-  private ClusterConnection connection;
+  private Connection connection;
 
-  public ZKTableArchiveClient(Configuration conf, ClusterConnection connection) {
+  public ZKTableArchiveClient(Configuration conf, Connection connection) {
     super(conf);
     this.connection = connection;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
index c3f8f8b..45dc8be 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
@@ -64,7 +64,7 @@ public interface AsyncClusterConnection extends AsyncConnection {
       List<Entry> entries, int replicaId, int numRetries, long operationTimeoutNs);
 
   /**
-   * Return all the replicas for a region. Used for regiong replica replication.
+   * Return all the replicas for a region. Used for region replica replication.
    */
   CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
       boolean reload);
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 b577d13..a405a12 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
@@ -1165,7 +1165,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (QuotaUtil.isQuotaEnabled(conf)) {
       // Create the quota snapshot notifier
       spaceQuotaSnapshotNotifier = createQuotaSnapshotNotifier();
-      spaceQuotaSnapshotNotifier.initialize(getClusterConnection());
+      spaceQuotaSnapshotNotifier.initialize(getConnection());
       this.quotaObserverChore = new QuotaObserverChore(this, getMasterMetrics());
       // Start the chore to read the region FS space reports and act on them
       getChoreService().scheduleChore(quotaObserverChore);
@@ -1262,7 +1262,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    */
   private boolean waitForNamespaceOnline() throws InterruptedException, IOException {
     TableState nsTableState =
-      MetaTableAccessor.getTableState(getClusterConnection(), TableName.NAMESPACE_TABLE_NAME);
+      MetaTableAccessor.getTableState(getConnection(), TableName.NAMESPACE_TABLE_NAME);
     if (nsTableState == null || nsTableState.isDisabled()) {
       // this means we have already migrated the data and disabled or deleted the namespace table,
       // or this is a new depliy which does not have a namespace table from the beginning.
@@ -1852,7 +1852,7 @@ public class HMaster extends HRegionServer implements MasterServices {
         List<NormalizationPlan> plans = this.normalizer.computePlanForTable(table);
         if (plans != null) {
           for (NormalizationPlan plan : plans) {
-            plan.execute(clusterConnection.getAdmin());
+            plan.execute(connection.getAdmin());
             if (plan.getType() == PlanType.SPLIT) {
               splitPlanCount++;
             } else if (plan.getType() == PlanType.MERGE) {
@@ -3049,8 +3049,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     // this is what we want especially if the Master is in startup phase doing call outs to
     // hbase:meta, etc. when cluster is down. Without ths connection close, we'd have to wait on
     // the rpc to timeout.
-    if (this.clusterConnection != null) {
-      this.clusterConnection.close();
+    if (this.connection != null) {
+      this.connection.close();
     }
     if (this.asyncClusterConnection != null) {
       this.asyncClusterConnection.close();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManagerUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManagerUtil.java
index 97ae7ea..285fc62 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManagerUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManagerUtil.java
@@ -28,19 +28,16 @@ import java.util.stream.Stream;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
-
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
 
@@ -66,22 +63,15 @@ final class AssignmentManagerUtil {
   static GetRegionInfoResponse getRegionInfoResponse(final MasterProcedureEnv env,
       final ServerName regionLocation, final RegionInfo hri, boolean includeBestSplitRow)
       throws IOException {
-    // TODO: There is no timeout on this controller. Set one!
-    HBaseRpcController controller =
-      env.getMasterServices().getClusterConnection().getRpcControllerFactory().newController();
-    final AdminService.BlockingInterface admin =
-      env.getMasterServices().getClusterConnection().getAdmin(regionLocation);
+    AsyncRegionServerAdmin admin =
+      env.getMasterServices().getAsyncClusterConnection().getRegionServerAdmin(regionLocation);
     GetRegionInfoRequest request = null;
     if (includeBestSplitRow) {
       request = RequestConverter.buildGetRegionInfoRequest(hri.getRegionName(), false, true);
     } else {
       request = RequestConverter.buildGetRegionInfoRequest(hri.getRegionName());
     }
-    try {
-      return admin.getRegionInfo(controller, request);
-    } catch (ServiceException e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
+    return FutureUtils.get(admin.getRegionInfo(request));
   }
 
   private static void lock(List<RegionStateNode> regionNodes) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
index 3972700..b9797bc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
@@ -90,7 +90,7 @@ public class RegionServerSpaceQuotaManager {
       return;
     }
     // Start the chores
-    this.spaceQuotaRefresher = new SpaceQuotaRefresherChore(this, rsServices.getClusterConnection());
+    this.spaceQuotaRefresher = new SpaceQuotaRefresherChore(this, rsServices.getConnection());
     rsServices.getChoreService().scheduleChore(spaceQuotaRefresher);
     this.regionSizeReporter = new RegionSizeReportingChore(rsServices);
     rsServices.getChoreService().scheduleChore(regionSizeReporter);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java
index 9d24c92..eb2e27a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java
@@ -17,17 +17,17 @@
 package org.apache.hadoop.hbase.quotas.policies;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
 import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
 import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A {@link SpaceViolationPolicyEnforcement} which disables the table. The enforcement
@@ -44,7 +44,9 @@ public class DisableTableViolationPolicyEnforcement extends DefaultViolationPoli
       if (LOG.isTraceEnabled()) {
         LOG.trace("Starting disable of " + getTableName());
       }
-      getRegionServerServices().getClusterConnection().getAdmin().disableTable(getTableName());
+      try (Admin admin = getRegionServerServices().getConnection().getAdmin()) {
+        admin.disableTable(getTableName());
+      }
       if (LOG.isTraceEnabled()) {
         LOG.trace("Disable is complete for " + getTableName());
       }
@@ -59,7 +61,9 @@ public class DisableTableViolationPolicyEnforcement extends DefaultViolationPoli
       if (LOG.isTraceEnabled()) {
         LOG.trace("Starting enable of " + getTableName());
       }
-      getRegionServerServices().getClusterConnection().getAdmin().enableTable(getTableName());
+      try (Admin admin = getRegionServerServices().getConnection().getAdmin()) {
+        admin.enableTable(getTableName());
+      }
       if (LOG.isTraceEnabled()) {
         LOG.trace("Enable is complete for " + getTableName());
       }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 710efc1..d15a11c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -83,7 +83,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.ZNodeClearer;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionUtils;
@@ -259,14 +258,17 @@ public class HRegionServer extends HasThread implements
   protected HeapMemoryManager hMemManager;
 
   /**
-   * Cluster connection to be shared by services.
+   * Connection to be shared by services.
+   * <p/>
    * Initialized at server startup and closed when server shuts down.
+   * <p/>
    * Clients must never close it explicitly.
-   * Clients hosted by this Server should make use of this clusterConnection rather than create
-   * their own; if they create their own, there is no way for the hosting server to shutdown
-   * ongoing client RPCs.
+   * <p/>
+   * Clients hosted by this Server should make use of this connection rather than create their own;
+   * if they create their own, there is no way for the hosting server to shutdown ongoing client
+   * RPCs.
    */
-  protected ClusterConnection clusterConnection;
+  protected Connection connection;
 
   /**
    * The asynchronous cluster connection to be shared by services.
@@ -805,11 +807,11 @@ public class HRegionServer extends HasThread implements
    * Create a 'smarter' Connection, one that is capable of by-passing RPC if the request is to the
    * local server; i.e. a short-circuit Connection. Safe to use going to local or remote server.
    */
-  private ClusterConnection createClusterConnection() throws IOException {
+  private Connection createConnection() throws IOException {
     // Create a cluster connection that when appropriate, can short-circuit and go directly to the
     // local server if the request is to the local server bypassing RPC. Can be used for both local
     // and remote invocations.
-    ClusterConnection conn =
+    Connection conn =
       ConnectionUtils.createShortCircuitConnection(unsetClientZookeeperQuorum(), null,
         userProvider.getCurrent(), serverName, rpcServices, rpcServices);
     // This is used to initialize the batch thread pool inside the connection implementation.
@@ -846,8 +848,8 @@ public class HRegionServer extends HasThread implements
    * Setup our cluster connection if not already initialized.
    */
   protected final synchronized void setupClusterConnection() throws IOException {
-    if (clusterConnection == null) {
-      clusterConnection = createClusterConnection();
+    if (connection == null) {
+      connection = createConnection();
       asyncClusterConnection =
         ClusterConnectionFactory.createAsyncClusterConnection(unsetClientZookeeperQuorum(),
           new InetSocketAddress(this.rpcServices.isa.getAddress(), 0), userProvider.getCurrent());
@@ -1119,9 +1121,9 @@ public class HRegionServer extends HasThread implements
       LOG.info("stopping server " + this.serverName);
     }
 
-    if (this.clusterConnection != null && !clusterConnection.isClosed()) {
+    if (this.connection != null && !connection.isClosed()) {
       try {
-        this.clusterConnection.close();
+        this.connection.close();
       } catch (IOException e) {
         // Although the {@link Closeable} interface throws an {@link
         // IOException}, in reality, the implementation would never do that.
@@ -2193,12 +2195,7 @@ public class HRegionServer extends HasThread implements
 
   @Override
   public Connection getConnection() {
-    return getClusterConnection();
-  }
-
-  @Override
-  public ClusterConnection getClusterConnection() {
-    return this.clusterConnection;
+    return this.connection;
   }
 
   @Override
@@ -2301,7 +2298,7 @@ public class HRegionServer extends HasThread implements
           }
         } else {
           try {
-            MetaTableAccessor.updateRegionLocation(clusterConnection,
+            MetaTableAccessor.updateRegionLocation(connection,
               hris[0], serverName, openSeqNum, masterSystemTime);
           } catch (IOException e) {
             LOG.info("Failed to update meta", e);
@@ -2329,7 +2326,7 @@ public class HRegionServer extends HasThread implements
     // Keep looping till we get an error. We want to send reports even though server is going down.
     // Only go down if clusterConnection is null. It is set to null almost as last thing as the
     // HRegionServer does down.
-    while (this.clusterConnection != null && !this.clusterConnection.isClosed()) {
+    while (this.connection != null && !this.connection.isClosed()) {
       RegionServerStatusService.BlockingInterface rss = rssStub;
       try {
         if (rss == null) {
@@ -3782,7 +3779,7 @@ public class HRegionServer extends HasThread implements
 
   @Override
   public void unassign(byte[] regionName) throws IOException {
-    clusterConnection.getAdmin().unassign(regionName, false);
+    connection.getAdmin().unassign(regionName, false);
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
index a960c31..67e945f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
@@ -207,7 +207,7 @@ public class DumpReplicationQueues extends Configured implements Tool {
 
     Configuration conf = getConf();
     HBaseAdmin.available(conf);
-    ClusterConnection connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
+    Connection connection = ConnectionFactory.createConnection(conf);
     Admin admin = connection.getAdmin();
 
     ZKWatcher zkw = new ZKWatcher(conf, "DumpReplicationQueues" + System.currentTimeMillis(),
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index 7d1245c..bbd7675 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -153,7 +152,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
     }
 
     @Override
-    public ClusterConnection getConnection() {
+    public Connection getConnection() {
       return null;
     }
 
@@ -163,11 +162,6 @@ public class ReplicationSyncUp extends Configured implements Tool {
     }
 
     @Override
-    public ClusterConnection getClusterConnection() {
-      return null;
-    }
-
-    @Override
     public FileSystem getFileSystem() {
       return null;
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index dfaa16d..3c84da0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -89,7 +89,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
@@ -161,9 +160,6 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Ordering;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hbase.thirdparty.com.google.common.collect.TreeMultimap;
 
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
-
 /**
  * HBaseFsck (hbck) is a tool for checking and repairing region consistency and
  * table integrity problems in a corrupted HBase. This tool was written for hbase-1.x. It does not
@@ -245,7 +241,7 @@ public class HBaseFsck extends Configured implements Closeable {
    **********************/
   private static final Logger LOG = LoggerFactory.getLogger(HBaseFsck.class.getName());
   private ClusterMetrics status;
-  private ClusterConnection connection;
+  private Connection connection;
   private Admin admin;
   private Table meta;
   // threads to do ||izable tasks: retrieve data from regionservers, handle overlapping regions
@@ -585,7 +581,7 @@ public class HBaseFsck extends Configured implements Closeable {
 
     LOG.info("Launching hbck");
 
-    connection = (ClusterConnection)ConnectionFactory.createConnection(getConf());
+    connection = ConnectionFactory.createConnection(getConf());
     admin = connection.getAdmin();
     meta = connection.getTable(TableName.META_TABLE_NAME);
     status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS,
@@ -4332,10 +4328,10 @@ public class HBaseFsck extends Configured implements Closeable {
     private final HBaseFsck hbck;
     private final ServerName rsinfo;
     private final ErrorReporter errors;
-    private final ClusterConnection connection;
+    private final Connection connection;
 
     WorkItemRegion(HBaseFsck hbck, ServerName info,
-                   ErrorReporter errors, ClusterConnection connection) {
+                   ErrorReporter errors, Connection connection) {
       this.hbck = hbck;
       this.rsinfo = info;
       this.errors = errors;
@@ -4346,32 +4342,29 @@ public class HBaseFsck extends Configured implements Closeable {
     public synchronized Void call() throws IOException {
       errors.progress();
       try {
-        BlockingInterface server = connection.getAdmin(rsinfo);
-
         // list all online regions from this region server
-        List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(server);
+        List<RegionInfo> regions = connection.getAdmin().getRegions(rsinfo);
         regions = filterRegions(regions);
 
         if (details) {
-          errors.detail("RegionServer: " + rsinfo.getServerName() +
-                           " number of regions: " + regions.size());
-          for (RegionInfo rinfo: regions) {
-            errors.detail("  " + rinfo.getRegionNameAsString() +
-                             " id: " + rinfo.getRegionId() +
-                             " encoded_name: " + rinfo.getEncodedName() +
-                             " start: " + Bytes.toStringBinary(rinfo.getStartKey()) +
-                             " end: " + Bytes.toStringBinary(rinfo.getEndKey()));
+          errors.detail(
+            "RegionServer: " + rsinfo.getServerName() + " number of regions: " + regions.size());
+          for (RegionInfo rinfo : regions) {
+            errors.detail("  " + rinfo.getRegionNameAsString() + " id: " + rinfo.getRegionId() +
+              " encoded_name: " + rinfo.getEncodedName() + " start: " +
+              Bytes.toStringBinary(rinfo.getStartKey()) + " end: " +
+              Bytes.toStringBinary(rinfo.getEndKey()));
           }
         }
 
         // check to see if the existence of this region matches the region in META
-        for (RegionInfo r:regions) {
+        for (RegionInfo r : regions) {
           HbckInfo hbi = hbck.getOrCreateInfo(r.getEncodedName());
           hbi.addServer(r, rsinfo);
         }
-      } catch (IOException e) {          // unable to connect to the region server.
-        errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE, "RegionServer: " + rsinfo.getServerName() +
-          " Unable to fetch region information. " + e);
+      } catch (IOException e) { // unable to connect to the region server.
+        errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE,
+          "RegionServer: " + rsinfo.getServerName() + " Unable to fetch region information. " + e);
         throw e;
       }
       return null;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
index 121d06c..d4a28c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
index 5805793..d095fa3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
@@ -26,19 +26,17 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Provides ability to create multiple Connection instances and allows to process a batch of
@@ -112,14 +110,11 @@ public class MultiHConnection {
    * @param callback to run when results are in
    * @throws IOException If IO failure occurs
    */
-  @SuppressWarnings("deprecation")
   public <R> void processBatchCallback(List<? extends Row> actions, TableName tableName,
       Object[] results, Batch.Callback<R> callback) throws IOException {
     // Currently used by RegionStateStore
-    ClusterConnection conn =
-      (ClusterConnection) connections[ThreadLocalRandom.current().nextInt(noOfConnections)];
-
-    HTable.doBatchWithCallback(actions, results, callback, conn, batchPool, tableName);
+    HTable.doBatchWithCallback(actions, results, callback,
+      connections[ThreadLocalRandom.current().nextInt(noOfConnections)], batchPool, tableName);
   }
 
   // Copied from ConnectionImplementation.getBatchPool()
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
index a779d36..540e7f3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
@@ -50,6 +49,7 @@ 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.NoServerForRegionException;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -550,7 +550,7 @@ public class RegionSplitter {
                   }
 
                   // make sure this region wasn't already split
-                  byte[] sk = regionLoc.getRegionInfo().getStartKey();
+                  byte[] sk = regionLoc.getRegion().getStartKey();
                   if (sk.length != 0) {
                     if (Bytes.equals(split, sk)) {
                       LOG.debug("Region already split on "
@@ -712,7 +712,6 @@ public class RegionSplitter {
       htd = table.getDescriptor();
     }
     try (RegionLocator regionLocator = connection.getRegionLocator(tableName)) {
-
       // for every region that hasn't been verified as a finished split
       for (Pair<byte[], byte[]> region : regionList) {
         byte[] start = region.getFirst();
@@ -720,7 +719,7 @@ public class RegionSplitter {
 
         // see if the new split daughter region has come online
         try {
-          HRegionInfo dri = regionLocator.getRegionLocation(split).getRegionInfo();
+          RegionInfo dri = regionLocator.getRegionLocation(split, true).getRegion();
           if (dri.isOffline() || !Bytes.equals(dri.getStartKey(), split)) {
             logicalSplitting.add(region);
             continue;
@@ -735,10 +734,10 @@ public class RegionSplitter {
         try {
           // when a daughter region is opened, a compaction is triggered
           // wait until compaction completes for both daughter regions
-          LinkedList<HRegionInfo> check = Lists.newLinkedList();
-          check.add(regionLocator.getRegionLocation(start).getRegionInfo());
-          check.add(regionLocator.getRegionLocation(split).getRegionInfo());
-          for (HRegionInfo hri : check.toArray(new HRegionInfo[check.size()])) {
+          LinkedList<RegionInfo> check = Lists.newLinkedList();
+          check.add(regionLocator.getRegionLocation(start).getRegion());
+          check.add(regionLocator.getRegionLocation(split).getRegion());
+          for (RegionInfo hri : check.toArray(new RegionInfo[check.size()])) {
             byte[] sk = hri.getStartKey();
             if (sk.length == 0)
               sk = splitAlgo.firstRow();
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index d3eaa77..4e26886 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -261,7 +261,7 @@ if ( fqtn != null ) {
         stateMap.put(regionInfo.getEncodedName(), regionState);
     }
   }
-  RegionLocator r = master.getClusterConnection().getRegionLocator(table.getName());
+  RegionLocator r = master.getConnection().getRegionLocator(table.getName());
   try { %>
 <h2>Table Attributes</h2>
 <table class="table table-striped">
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
index d760a7d..8020553 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase;
 
 import java.io.Closeable;
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
@@ -28,10 +27,6 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
-
 /**
  * This class defines methods that can help with managing HBase clusters
  * from unit tests and system tests. There are 3 types of cluster deployments:
@@ -98,24 +93,6 @@ public abstract class HBaseCluster implements Closeable, Configurable {
   }
 
   /**
-   * Returns an {@link MasterService.BlockingInterface} to the active master
-   */
-  public abstract MasterService.BlockingInterface getMasterAdminService()
-  throws IOException;
-
-  /**
-   * Returns an AdminProtocol interface to the regionserver
-   */
-  public abstract AdminService.BlockingInterface getAdminProtocol(ServerName serverName)
-  throws IOException;
-
-  /**
-   * Returns a ClientProtocol interface to the regionserver
-   */
-  public abstract ClientService.BlockingInterface getClientProtocol(ServerName serverName)
-  throws IOException;
-
-  /**
    * Starts a new region server on the given hostname or if this is a mini/local cluster,
    * starts a region server locally.
    * @param hostname the hostname to start the regionserver on
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
index 473eb74..92cfddf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -42,9 +41,6 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
 
 /**
@@ -521,15 +517,6 @@ public class MiniHBaseCluster extends HBaseCluster {
    * Returns the current active master, if available.
    * @return the active HMaster, null if none is active.
    */
-  @Override
-  public MasterService.BlockingInterface getMasterAdminService() {
-    return this.hbaseCluster.getActiveMaster().getMasterRpcServices();
-  }
-
-  /**
-   * Returns the current active master, if available.
-   * @return the active HMaster, null if none is active.
-   */
   public HMaster getMaster() {
     return this.hbaseCluster.getActiveMaster();
   }
@@ -921,15 +908,4 @@ public class MiniHBaseCluster extends HBaseCluster {
     }
     return -1;
   }
-
-  @Override
-  public AdminService.BlockingInterface getAdminProtocol(ServerName serverName) throws IOException {
-    return getRegionServer(getRegionServerIndex(serverName)).getRSRpcServices();
-  }
-
-  @Override
-  public ClientService.BlockingInterface getClientProtocol(ServerName serverName)
-  throws IOException {
-    return getRegionServer(getRegionServerIndex(serverName)).getRSRpcServices();
-  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index 5205960..f3e8d12 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -32,7 +32,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.locking.EntityLock;
@@ -160,7 +159,7 @@ public class MockRegionServerServices implements RegionServerServices {
   }
 
   @Override
-  public ClusterConnection getConnection() {
+  public Connection getConnection() {
     return null;
   }
 
@@ -264,7 +263,6 @@ public class MockRegionServerServices implements RegionServerServices {
 
   @Override
   public ServerNonceManager getNonceManager() {
-    // TODO Auto-generated method stub
     return null;
   }
 
@@ -275,7 +273,6 @@ public class MockRegionServerServices implements RegionServerServices {
 
   @Override
   public boolean registerService(Service service) {
-    // TODO Auto-generated method stub
     return false;
   }
 
@@ -290,11 +287,6 @@ public class MockRegionServerServices implements RegionServerServices {
   }
 
   @Override
-  public ClusterConnection getClusterConnection() {
-    return null;
-  }
-
-  @Override
   public ThroughputController getFlushThroughputController() {
     return null;
   }
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 16f3930..618fe74 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
@@ -35,9 +35,9 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 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.Put;
 import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
@@ -86,7 +86,7 @@ public class TestZooKeeperTableArchiveClient {
   private static final byte[] TABLE_NAME = Bytes.toBytes(STRING_TABLE_NAME);
   private static ZKTableArchiveClient archivingClient;
   private final List<Path> toCleanup = new ArrayList<>();
-  private static ClusterConnection CONNECTION;
+  private static Connection CONNECTION;
   private static RegionServerServices rss;
 
   /**
@@ -96,7 +96,7 @@ public class TestZooKeeperTableArchiveClient {
   public static void setupCluster() throws Exception {
     setupConf(UTIL.getConfiguration());
     UTIL.startMiniZKCluster();
-    CONNECTION = (ClusterConnection)ConnectionFactory.createConnection(UTIL.getConfiguration());
+    CONNECTION = ConnectionFactory.createConnection(UTIL.getConfiguration());
     archivingClient = new ZKTableArchiveClient(UTIL.getConfiguration(), CONNECTION);
     // make hfile archiving node so we can archive files
     ZKWatcher watcher = UTIL.getZooKeeperWatcher();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
index 2a5a395..fe86dde 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
@@ -57,11 +57,11 @@ public class HConnectionTestingUtility {
    * @throws ZooKeeperConnectionException
    */
   public static ConnectionImplementation getMockedConnection(final Configuration conf)
-  throws ZooKeeperConnectionException {
+      throws ZooKeeperConnectionException {
     ConnectionImplementation connection = Mockito.mock(ConnectionImplementation.class);
     Mockito.when(connection.getConfiguration()).thenReturn(conf);
-    Mockito.when(connection.getRpcControllerFactory()).thenReturn(
-      Mockito.mock(RpcControllerFactory.class));
+    Mockito.when(connection.getRpcControllerFactory())
+      .thenReturn(Mockito.mock(RpcControllerFactory.class));
     // we need a real retrying caller
     RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf);
     Mockito.when(connection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
@@ -81,11 +81,10 @@ public class HConnectionTestingUtility {
    *          the mocked connection
    * @return Mock up a connection that returns a {@link Configuration} when
    *         {@link ConnectionImplementation#getConfiguration()} is called, a 'location' when
-   *         {@link ConnectionImplementation#getRegionLocation(TableName,byte[], boolean)}
-   *         is called, and that returns the passed
-   *         {@link AdminProtos.AdminService.BlockingInterface} instance when
-   *         {@link ConnectionImplementation#getAdmin(ServerName)} is called, returns the passed
-   *         {@link ClientProtos.ClientService.BlockingInterface} instance when
+   *         {@link ConnectionImplementation#getRegionLocation(TableName, byte[], boolean)} is
+   *         called, and that returns the passed {@link AdminProtos.AdminService.BlockingInterface}
+   *         instance when {@link ConnectionImplementation#getAdmin(ServerName)} is called, returns
+   *         the passed {@link ClientProtos.ClientService.BlockingInterface} instance when
    *         {@link ConnectionImplementation#getClient(ServerName)} is called (Be sure to call
    *         {@link Connection#close()} when done with this mocked Connection.
    */
@@ -138,9 +137,14 @@ public class HConnectionTestingUtility {
    * calling {@link Connection#close()} else it will stick around; this is probably not what you
    * want.
    * @param conf configuration
+<<<<<<< HEAD
    * @return ConnectionImplementation object for <code>conf</code>
    * @throws ZooKeeperConnectionException [Dead link]: See also
    *           {http://mockito.googlecode.com/svn/branches/1.6/javadoc/org/mockito/Mockito.html#spy(T)}
+=======
+   * @return ClusterConnection object for <code>conf</code> [Dead link]: See also
+   *         {http://mockito.googlecode.com/svn/branches/1.6/javadoc/org/mockito/Mockito.html#spy(T)}
+>>>>>>> 7e889e40e8... tmp
    */
   public static ConnectionImplementation getSpiedConnection(final Configuration conf)
       throws IOException {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index 376aa92..1195887 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -632,8 +632,8 @@ public class TestAdmin1 {
     assertFalse(this.admin.tableExists(tableName));
   }
 
-  protected void verifyRoundRobinDistribution(ClusterConnection c, RegionLocator regionLocator, int
-      expectedRegions) throws IOException {
+  private void verifyRoundRobinDistribution(ConnectionImplementation c,
+      RegionLocator regionLocator, int expectedRegions) throws IOException {
     int numRS = c.getCurrentNrHRS();
     List<HRegionLocation> regions = regionLocator.getAllRegionLocations();
     Map<ServerName, List<RegionInfo>> server2Regions = new HashMap<>();
@@ -652,13 +652,14 @@ public class TestAdmin1 {
       // which contains less regions by intention.
       numRS--;
     }
-    float average = (float) expectedRegions/numRS;
-    int min = (int)Math.floor(average);
-    int max = (int)Math.ceil(average);
+    float average = (float) expectedRegions / numRS;
+    int min = (int) Math.floor(average);
+    int max = (int) Math.ceil(average);
     for (List<RegionInfo> regionList : server2Regions.values()) {
-      assertTrue("numRS=" + numRS + ", min=" + min + ", max=" + max +
-        ", size=" + regionList.size() + ", tablesOnMaster=" + tablesOnMaster,
-      regionList.size() == min || regionList.size() == max);
+      assertTrue(
+        "numRS=" + numRS + ", min=" + min + ", max=" + max + ", size=" + regionList.size() +
+          ", tablesOnMaster=" + tablesOnMaster,
+        regionList.size() == min || regionList.size() == max);
     }
   }
 
@@ -739,7 +740,7 @@ public class TestAdmin1 {
     List<HRegionLocation> regions;
     Iterator<HRegionLocation> hris;
     RegionInfo hri;
-    ClusterConnection conn = (ClusterConnection) TEST_UTIL.getConnection();
+    ConnectionImplementation conn = (ConnectionImplementation) TEST_UTIL.getConnection();
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
       regions = l.getAllRegionLocations();
 
@@ -1257,13 +1258,9 @@ public class TestAdmin1 {
       byte[][] nameofRegionsToMerge = new byte[2][];
       nameofRegionsToMerge[0] =  regions.get(1).getFirst().getEncodedNameAsBytes();
       nameofRegionsToMerge[1] = regions.get(2).getFirst().getEncodedNameAsBytes();
-      MergeTableRegionsRequest request = RequestConverter
-          .buildMergeTableRegionsRequest(
-            nameofRegionsToMerge,
-            true,
-            HConstants.NO_NONCE,
-            HConstants.NO_NONCE);
-      ((ClusterConnection) TEST_UTIL.getAdmin().getConnection()).getMaster()
+      MergeTableRegionsRequest request = RequestConverter.buildMergeTableRegionsRequest(
+        nameofRegionsToMerge, true, HConstants.NO_NONCE, HConstants.NO_NONCE);
+      ((ConnectionImplementation) TEST_UTIL.getAdmin().getConnection()).getMaster()
         .mergeTableRegions(null, request);
     } catch (org.apache.hbase.thirdparty.com.google.protobuf.ServiceException m) {
       Throwable t = m.getCause();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index 2c14eaf..19dfd30 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -783,7 +783,7 @@ public class TestAdmin2 {
     Assert.assertNotNull(store);
     Assert.assertEquals(expectedStoreFilesSize, store.getSize());
 
-    ClusterConnection conn = ((ClusterConnection) admin.getConnection());
+    ConnectionImplementation conn = (ConnectionImplementation) admin.getConnection();
     HBaseRpcController controller = conn.getRpcControllerFactory().newController();
     for (int i = 0; i < 10; i++) {
       RegionInfo ri =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
index 990eed4..38f7aab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
@@ -274,7 +274,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
 
   private void verifyRoundRobinDistribution(List<HRegionLocation> regions, int expectedRegions)
       throws IOException {
-    int numRS = ((ClusterConnection) TEST_UTIL.getConnection()).getCurrentNrHRS();
+    int numRS = ((ConnectionImplementation) TEST_UTIL.getConnection()).getCurrentNrHRS();
 
     Map<ServerName, List<RegionInfo>> server2Regions = new HashMap<>();
     regions.stream().forEach((loc) -> {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java
index cd27a30..fd0eb7b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java
@@ -94,9 +94,9 @@ public class TestCISleep extends AbstractTestCITimeout {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.createTable(tableName, FAM_NAM);
     ClientServiceCallable<Object> regionServerCallable =
-      new ClientServiceCallable<Object>(TEST_UTIL.getConnection(), tableName, FAM_NAM,
-          new RpcControllerFactory(TEST_UTIL.getConfiguration()).newController(),
-          HConstants.PRIORITY_UNSET) {
+      new ClientServiceCallable<Object>((ConnectionImplementation) TEST_UTIL.getConnection(),
+        tableName, FAM_NAM, new RpcControllerFactory(TEST_UTIL.getConfiguration()).newController(),
+        HConstants.PRIORITY_UNSET) {
         @Override
         protected Object rpcCall() throws Exception {
           return null;
@@ -126,9 +126,9 @@ public class TestCISleep extends AbstractTestCITimeout {
       assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f));
     }
 
-    try (
-      MasterCallable<Object> masterCallable = new MasterCallable<Object>(TEST_UTIL.getConnection(),
-          new RpcControllerFactory(TEST_UTIL.getConfiguration())) {
+    try (MasterCallable<Object> masterCallable =
+      new MasterCallable<Object>((ConnectionImplementation) TEST_UTIL.getConnection(),
+        new RpcControllerFactory(TEST_UTIL.getConfiguration())) {
         @Override
         protected Object rpcCall() throws Exception {
           return null;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
index d6f32f5..ae217cd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
@@ -94,7 +94,8 @@ public class TestClientPushback {
   public void testClientTracksServerPushback() throws Exception{
     Configuration conf = UTIL.getConfiguration();
 
-    ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
+    ConnectionImplementation conn =
+      (ConnectionImplementation) ConnectionFactory.createConnection(conf);
     BufferedMutatorImpl mutator = (BufferedMutatorImpl) conn.getBufferedMutator(tableName);
 
     HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0);
@@ -182,7 +183,8 @@ public class TestClientPushback {
   @Test
   public void testMutateRowStats() throws IOException {
     Configuration conf = UTIL.getConfiguration();
-    ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
+    ConnectionImplementation conn =
+      (ConnectionImplementation) ConnectionFactory.createConnection(conf);
     Table table = conn.getTable(tableName);
     HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0);
     Region region = rs.getRegions(tableName).get(0);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
index 9523a61..76b83fe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
@@ -837,7 +837,7 @@ public class TestConnectionImplementation {
    *  from ZK by the client.
    */
   @Test
-  public void testConnection() throws Exception{
+  public void testConnection() throws Exception {
     // We create an empty config and add the ZK address.
     Configuration c = new Configuration();
     c.set(HConstants.ZOOKEEPER_QUORUM,
@@ -846,7 +846,8 @@ public class TestConnectionImplementation {
       TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_CLIENT_PORT));
 
     // This should be enough to connect
-    ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(c);
+    ConnectionImplementation conn =
+      (ConnectionImplementation) ConnectionFactory.createConnection(c);
     assertTrue(conn.isMasterRunning());
     conn.close();
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index 1315d4a..ff151fe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
@@ -159,7 +159,7 @@ public class TestFromClientSide3 {
       // connection needed for poll-wait
       HRegionLocation loc = locator.getRegionLocation(row, true);
       AdminProtos.AdminService.BlockingInterface server =
-        ((ClusterConnection) admin.getConnection()).getAdmin(loc.getServerName());
+        ((ConnectionImplementation) admin.getConnection()).getAdmin(loc.getServerName());
       byte[] regName = loc.getRegionInfo().getRegionName();
 
       for (int i = 0; i < nFlushes; i++) {
@@ -289,7 +289,7 @@ public class TestFromClientSide3 {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10);
     Admin admin = TEST_UTIL.getAdmin();
-    ClusterConnection connection = (ClusterConnection) TEST_UTIL.getConnection();
+    ConnectionImplementation connection = (ConnectionImplementation) TEST_UTIL.getConnection();
 
     // Create 3 store files.
     byte[] row = Bytes.toBytes(random.nextInt());
@@ -681,7 +681,7 @@ public class TestFromClientSide3 {
 
   @Test
   public void testConnectionDefaultUsesCodec() throws Exception {
-    ClusterConnection con = (ClusterConnection) TEST_UTIL.getConnection();
+    ConnectionImplementation con = (ConnectionImplementation) TEST_UTIL.getConnection();
     assertTrue(con.hasCellBlockSupport());
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java
index 53f5064..108ab7f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java
@@ -107,15 +107,13 @@ public class TestMetaTableAccessorNoCluster {
     Result r = Result.create(kvs);
     assertNull(MetaTableAccessor.getRegionInfo(r));
 
-    byte [] f = HConstants.CATALOG_FAMILY;
+    byte[] f = HConstants.CATALOG_FAMILY;
     // Make a key value that doesn't have the expected qualifier.
-    kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
-      HConstants.SERVER_QUALIFIER, f));
+    kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f, HConstants.SERVER_QUALIFIER, f));
     r = Result.create(kvs);
     assertNull(MetaTableAccessor.getRegionInfo(r));
     // Make a key that does not have a regioninfo value.
-    kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
-      HConstants.REGIONINFO_QUALIFIER, f));
+    kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f, HConstants.REGIONINFO_QUALIFIER, f));
     RegionInfo hri = MetaTableAccessor.getRegionInfo(Result.create(kvs));
     assertTrue(hri == null);
     // OK, give it what it expects
@@ -161,7 +159,7 @@ public class TestMetaTableAccessorNoCluster {
           RegionInfo.toByteArray(RegionInfoBuilder.FIRST_META_REGIONINFO)));
       kvs.add(new KeyValue(rowToVerify,
         HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
-        Bytes.toBytes(sn.getHostAndPort())));
+        Bytes.toBytes(sn.getAddress().toString())));
       kvs.add(new KeyValue(rowToVerify,
         HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
         Bytes.toBytes(sn.getStartcode())));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index e5280a6..336a272 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -390,11 +390,9 @@ public class TestMetaWithReplicas {
 
   @Ignore @Test // Disabled. Relies on FSCK which needs work for AMv2.
   public void testHBaseFsckWithFewerMetaReplicas() throws Exception {
-    ClusterConnection c = (ClusterConnection)ConnectionFactory.createConnection(
-        TEST_UTIL.getConfiguration());
     RegionLocations rl = new RegionLocations(getMetaRegionLocations());
-    HBaseFsckRepair.closeRegionSilentlyAndWait(c,
-        rl.getRegionLocation(1).getServerName(), rl.getRegionLocation(1).getRegionInfo());
+    HBaseFsckRepair.closeRegionSilentlyAndWait(TEST_UTIL.getConnection(),
+      rl.getRegionLocation(1).getServerName(), rl.getRegionLocation(1).getRegionInfo());
     // check that problem exists
     HBaseFsck hbck = doFsck(TEST_UTIL.getConfiguration(), false);
     assertErrors(hbck, new ERROR_CODE[]{ERROR_CODE.UNKNOWN,ERROR_CODE.NO_META_REGION});
@@ -407,11 +405,9 @@ public class TestMetaWithReplicas {
 
   @Ignore @Test // The close silently doesn't work any more since HBASE-14614. Fix.
   public void testHBaseFsckWithFewerMetaReplicaZnodes() throws Exception {
-    ClusterConnection c = (ClusterConnection)ConnectionFactory.createConnection(
-        TEST_UTIL.getConfiguration());
     RegionLocations rl = new RegionLocations(getMetaRegionLocations());
-    HBaseFsckRepair.closeRegionSilentlyAndWait(c,
-        rl.getRegionLocation(2).getServerName(), rl.getRegionLocation(2).getRegionInfo());
+    HBaseFsckRepair.closeRegionSilentlyAndWait(TEST_UTIL.getConnection(),
+      rl.getRegionLocation(2).getServerName(), rl.getRegionLocation(2).getRegionInfo());
     ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
     ZKUtil.deleteNode(zkw, zkw.getZNodePaths().getZNodeForReplica(2));
     // check that problem exists
@@ -485,20 +481,17 @@ public class TestMetaWithReplicas {
   public void testShutdownOfReplicaHolder() throws Exception {
     // checks that the when the server holding meta replica is shut down, the meta replica
     // can be recovered
-    try (ClusterConnection conn = (ClusterConnection)
-        ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
-      HRegionLocation hrl = getMetaRegionLocations().get(1);
-      ServerName oldServer = hrl.getServerName();
-      TEST_UTIL.getHBaseClusterInterface().killRegionServer(oldServer);
-      int i = 0;
-      do {
-        LOG.debug("Waiting for the replica " + hrl.getRegionInfo() + " to come up");
-        Thread.sleep(10000); //wait for the detection/recovery
-        hrl = getMetaRegionLocations().get(1);
-        i++;
-      } while ((hrl == null || hrl.getServerName().equals(oldServer)) && i < 3);
-      assertTrue(i != 3);
-    }
+    HRegionLocation hrl = getMetaRegionLocations().get(1);
+    ServerName oldServer = hrl.getServerName();
+    TEST_UTIL.getHBaseClusterInterface().killRegionServer(oldServer);
+    int i = 0;
+    do {
+      LOG.debug("Waiting for the replica " + hrl.getRegionInfo() + " to come up");
+      Thread.sleep(10000); // wait for the detection/recovery
+      hrl = getMetaRegionLocations().get(1);
+      i++;
+    } while ((hrl == null || hrl.getServerName().equals(oldServer)) && i < 3);
+    assertTrue(i != 3);
   }
 
   @Ignore @Test // Disabled because fsck and this needs work for AMv2
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
index 7d36e99..50c9bd8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
@@ -553,7 +553,7 @@ public class TestMultiParallel {
     };
 
     NonceGenerator oldCnm =
-      ConnectionUtils.injectNonceGeneratorForTesting((ClusterConnection)connection, cnm);
+      ConnectionUtils.injectNonceGeneratorForTesting((ConnectionImplementation) connection, cnm);
 
     // First test sequential requests.
     try {
@@ -615,7 +615,8 @@ public class TestMultiParallel {
       validateResult(result, QUALIFIER, Bytes.toBytes((numRequests / 2) + 1L));
       table.close();
     } finally {
-      ConnectionImplementation.injectNonceGeneratorForTesting((ClusterConnection) connection, oldCnm);
+      ConnectionImplementation.injectNonceGeneratorForTesting((ConnectionImplementation) connection,
+        oldCnm);
     }
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
index 6616b3b..c8a7ca1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
@@ -571,7 +571,7 @@ public class TestReplicasClient {
       LOG.info("get works and is not stale done");
 
       //reset
-      ClusterConnection connection = (ClusterConnection) HTU.getConnection();
+      ConnectionImplementation connection = (ConnectionImplementation) HTU.getConnection();
       Counter hedgedReadOps = connection.getConnectionMetrics().hedgedReadOps;
       Counter hedgedReadWin = connection.getConnectionMetrics().hedgedReadWin;
       hedgedReadOps.dec(hedgedReadOps.getCount());
@@ -638,7 +638,7 @@ public class TestReplicasClient {
 
       Thread.sleep(1000 + REFRESH_PERIOD * 2);
 
-      AsyncProcess ap = ((ClusterConnection) HTU.getConnection()).getAsyncProcess();
+      AsyncProcess ap = ((ConnectionImplementation) HTU.getConnection()).getAsyncProcess();
 
       // Make primary slowdown
       SlowMeCopro.getPrimaryCdl().set(new CountDownLatch(1));
@@ -654,16 +654,14 @@ public class TestReplicasClient {
       gets.add(g);
       Object[] results = new Object[2];
 
-      int operationTimeout = ((ClusterConnection) HTU.getConnection()).getConnectionConfiguration().getOperationTimeout();
-      int readTimeout = ((ClusterConnection) HTU.getConnection()).getConnectionConfiguration().getReadRpcTimeout();
-      AsyncProcessTask task = AsyncProcessTask.newBuilder()
-              .setPool(HTable.getDefaultExecutor(HTU.getConfiguration()))
-              .setTableName(table.getName())
-              .setRowAccess(gets)
-              .setResults(results)
-              .setOperationTimeout(operationTimeout)
-              .setRpcTimeout(readTimeout)
-              .build();
+      int operationTimeout = ((ConnectionImplementation) HTU.getConnection())
+        .getConnectionConfiguration().getOperationTimeout();
+      int readTimeout = ((ConnectionImplementation) HTU.getConnection())
+        .getConnectionConfiguration().getReadRpcTimeout();
+      AsyncProcessTask task =
+        AsyncProcessTask.newBuilder().setPool(HTable.getDefaultExecutor(HTU.getConfiguration()))
+          .setTableName(table.getName()).setRowAccess(gets).setResults(results)
+          .setOperationTimeout(operationTimeout).setRpcTimeout(readTimeout).build();
       AsyncRequestFuture reqs = ap.submit(task);
       reqs.waitUntilDone();
       // verify we got the right results back
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
index 93c6b4d..b9c16f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
@@ -206,7 +206,7 @@ public class TestSeparateClientZKCluster {
     // create table
     Connection conn = TEST_UTIL.getConnection();
     Admin admin = conn.getAdmin();
-    HTable table = (HTable) conn.getTable(tn);
+    Table table = conn.getTable(tn);
     try {
       ColumnFamilyDescriptorBuilder cfDescBuilder =
           ColumnFamilyDescriptorBuilder.newBuilder(family);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java
index beaa59b..f743388 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java
@@ -75,7 +75,7 @@ public class TestShortCircuitConnection {
     htd.addFamily(hcd);
     UTIL.createTable(htd, null);
     HRegionServer regionServer = UTIL.getRSForFirstRegionInTable(tableName);
-    ClusterConnection connection = regionServer.getClusterConnection();
+    ConnectionImplementation connection = (ConnectionImplementation) regionServer.getConnection();
     Table tableIf = connection.getTable(tableName);
     assertTrue(tableIf instanceof HTable);
     HTable table = (HTable) tableIf;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 3ebad66..c7cd1c1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
@@ -161,7 +160,7 @@ public class MockNoopMasterServices implements MasterServices {
   }
 
   @Override
-  public ClusterConnection getConnection() {
+  public Connection getConnection() {
     return null;
   }
 
@@ -353,11 +352,6 @@ public class MockNoopMasterServices implements MasterServices {
   }
 
   @Override
-  public ClusterConnection getClusterConnection() {
-    return null;
-  }
-
-  @Override
   public LoadBalancer getLoadBalancer() {
     return null;
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 73d53c7..7255d2f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
@@ -303,7 +302,7 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
   }
 
   @Override
-  public ClusterConnection getConnection() {
+  public Connection getConnection() {
     return null;
   }
 
@@ -619,11 +618,6 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
   }
 
   @Override
-  public ClusterConnection getClusterConnection() {
-    return null;
-  }
-
-  @Override
   public ThroughputController getFlushThroughputController() {
     return null;
   }
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 77667a7..7f9605e 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
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -313,7 +312,7 @@ public class TestActiveMasterManager {
     }
 
     @Override
-    public ClusterConnection getConnection() {
+    public Connection getConnection() {
       return null;
     }
 
@@ -331,12 +330,6 @@ public class TestActiveMasterManager {
     }
 
     @Override
-    public ClusterConnection getClusterConnection() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
     public FileSystem getFileSystem() {
       return null;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
index a0aae32..0deea15 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
@@ -18,16 +18,12 @@
 package org.apache.hadoop.hbase.master;
 
 import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 import java.net.InetAddress;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClockOutOfSyncException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.ClassRule;
@@ -51,14 +47,7 @@ public class TestClockSkewDetection {
   @Test
   public void testClockSkewDetection() throws Exception {
     final Configuration conf = HBaseConfiguration.create();
-    ServerManager sm = new ServerManager(new MockNoopMasterServices(conf) {
-      @Override
-      public ClusterConnection getClusterConnection() {
-        ClusterConnection conn = mock(ClusterConnection.class);
-        when(conn.getRpcControllerFactory()).thenReturn(mock(RpcControllerFactory.class));
-        return conn;
-      }
-    });
+    ServerManager sm = new ServerManager(new MockNoopMasterServices(conf));
 
     LOG.debug("regionServerStartup 1");
     InetAddress ia1 = InetAddress.getLocalHost();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index e73ba75..4f03108 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.ServerMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -184,7 +184,7 @@ public class TestMasterNoCluster {
     // Insert a mock for the connection, use TESTUTIL.getConfiguration rather than
     // the conf from the master; the conf will already have an ClusterConnection
     // associate so the below mocking of a connection will fail.
-    final ClusterConnection mockedConnection = HConnectionTestingUtility.getMockedConnectionAndDecorate(
+    final Connection mockedConnection = HConnectionTestingUtility.getMockedConnectionAndDecorate(
         TESTUTIL.getConfiguration(), rs0, rs0, rs0.getServerName(),
         HRegionInfo.FIRST_META_REGIONINFO);
     HMaster master = new HMaster(conf) {
@@ -212,12 +212,7 @@ public class TestMasterNoCluster {
       }
 
       @Override
-      public ClusterConnection getConnection() {
-        return mockedConnection;
-      }
-
-      @Override
-      public ClusterConnection getClusterConnection() {
+      public Connection getConnection() {
         return mockedConnection;
       }
     };
@@ -281,7 +276,7 @@ public class TestMasterNoCluster {
       }
 
       @Override
-      public ClusterConnection getConnection() {
+      public Connection getConnection() {
         // Insert a mock for the connection, use TESTUTIL.getConfiguration rather than
         // the conf from the master; the conf will already have a Connection
         // associate so the below mocking of a connection will fail.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
index 56467cc..ef64c94 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
@@ -31,8 +31,8 @@ import org.apache.hadoop.hbase.ServerMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -87,7 +87,7 @@ public class MockMasterServices extends MockNoopMasterServices {
   private MasterProcedureEnv procedureEnv;
   private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
   private ProcedureStore procedureStore;
-  private final ClusterConnection connection;
+  private final Connection connection;
   private final LoadBalancer balancer;
   private final ServerManager serverManager;
 
@@ -284,7 +284,7 @@ public class MockMasterServices extends MockNoopMasterServices {
   }
 
   @Override
-  public ClusterConnection getConnection() {
+  public Connection getConnection() {
     return this.connection;
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index c5fad32..3d6466d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -228,7 +227,7 @@ public class TestHFileCleaner {
     }
 
     @Override
-    public ClusterConnection getConnection() {
+    public Connection getConnection() {
       return null;
     }
 
@@ -261,12 +260,6 @@ public class TestHFileCleaner {
     }
 
     @Override
-    public ClusterConnection getClusterConnection() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
     public FileSystem getFileSystem() {
       return null;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
index fd11ff8..82c8684 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -164,7 +163,7 @@ public class TestHFileLinkCleaner {
     }
 
     @Override
-    public ClusterConnection getConnection() {
+    public Connection getConnection() {
       return null;
     }
 
@@ -193,13 +192,6 @@ public class TestHFileLinkCleaner {
     public ChoreService getChoreService() {
       return null;
     }
-
-    @Override
-    public ClusterConnection getClusterConnection() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
     @Override
     public FileSystem getFileSystem() {
       return null;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 6a5fe9c..7434b88 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -363,7 +362,7 @@ public class TestLogsCleaner {
     }
 
     @Override
-    public ClusterConnection getConnection() {
+    public Connection getConnection() {
       return null;
     }
 
@@ -394,11 +393,6 @@ public class TestLogsCleaner {
     }
 
     @Override
-    public ClusterConnection getClusterConnection() {
-      return null;
-    }
-
-    @Override
     public FileSystem getFileSystem() {
       return null;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index 9791643..b16d377 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -252,7 +251,7 @@ public class TestReplicationHFileCleaner {
     }
 
     @Override
-    public ClusterConnection getConnection() {
+    public Connection getConnection() {
       return null;
     }
 
@@ -285,12 +284,6 @@ public class TestReplicationHFileCleaner {
     }
 
     @Override
-    public ClusterConnection getClusterConnection() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
     public FileSystem getFileSystem() {
       return null;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index b4d16c6..29e6dbb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -359,11 +359,11 @@ public class MasterProcedureTestingUtility {
   //  Procedure Helpers
   // ==========================================================================
   public static long generateNonceGroup(final HMaster master) {
-    return master.getClusterConnection().getNonceGenerator().getNonceGroup();
+    return master.getAsyncClusterConnection().getNonceGenerator().getNonceGroup();
   }
 
   public static long generateNonce(final HMaster master) {
-    return master.getClusterConnection().getNonceGenerator().newNonce();
+    return master.getAsyncClusterConnection().getNonceGenerator().newNonce();
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index fd02cf4..af8cfb8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
@@ -48,8 +48,8 @@ import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ClientServiceCallable;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.RpcRetryingCaller;
@@ -66,7 +66,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener;
@@ -89,10 +88,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
-
 /**
  * Tests bulk loading of HFiles and shows the atomicity or lack of atomicity of
  * the region server's bullkLoad functionality.
@@ -214,29 +209,17 @@ public class TestHRegionServerBulkLoad {
       }
       // bulk load HFiles
       BulkLoadHFiles.create(UTIL.getConfiguration()).bulkLoad(tableName, family2Files);
+      final Connection conn = UTIL.getConnection();
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(conf);
         RpcRetryingCaller<Void> caller = factory.<Void> newCaller();
         // 5 * 50 = 250 open file handles!
-        ClientServiceCallable<Void> callable =
-          new ClientServiceCallable<Void>(UTIL.getConnection(), tableName, Bytes.toBytes("aaa"),
-            new RpcControllerFactory(UTIL.getConfiguration()).newController(),
-            HConstants.PRIORITY_UNSET) {
-            @Override
-            protected Void rpcCall() throws Exception {
-              LOG.debug(
-                "compacting " + getLocation() + " for row " + Bytes.toStringBinary(getRow()));
-              AdminProtos.AdminService.BlockingInterface server =
-                ((ClusterConnection) UTIL.getConnection()).getAdmin(getLocation().getServerName());
-              CompactRegionRequest request = RequestConverter.buildCompactRegionRequest(
-                getLocation().getRegionInfo().getRegionName(), true, null);
-              server.compactRegion(null, request);
-              numCompactions.incrementAndGet();
-              return null;
-            }
-          };
-        caller.callWithRetries(callable, Integer.MAX_VALUE);
+        try (RegionLocator locator = conn.getRegionLocator(tableName)) {
+          HRegionLocation loc = locator.getRegionLocation(Bytes.toBytes("aaa"), true);
+          conn.getAdmin().compactRegion(loc.getRegion().getRegionName());
+          numCompactions.incrementAndGet();
+        }
       }
     }
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
index 4a359e4..7c6598d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
@@ -829,7 +828,7 @@ public class TestHeapMemoryManager {
     }
 
     @Override
-    public ClusterConnection getConnection() {
+    public Connection getConnection() {
       return null;
     }
 
@@ -844,12 +843,6 @@ public class TestHeapMemoryManager {
     }
 
     @Override
-    public ClusterConnection getClusterConnection() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
     public FileSystem getFileSystem() {
       return null;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
index 43da846..b52bf19 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.SplitLogCounters;
 import org.apache.hadoop.hbase.SplitLogTask;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
 import org.apache.hadoop.hbase.executor.ExecutorService;
@@ -132,7 +131,7 @@ public class TestSplitLogWorker {
     }
 
     @Override
-    public ClusterConnection getConnection() {
+    public Connection getConnection() {
       return null;
     }
 
@@ -142,12 +141,6 @@ public class TestSplitLogWorker {
     }
 
     @Override
-    public ClusterConnection getClusterConnection() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
     public FileSystem getFileSystem() {
       return null;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 9e9d1d6..82b2843 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
@@ -470,7 +469,7 @@ public class TestWALLockup {
     }
 
     @Override
-    public ClusterConnection getConnection() {
+    public Connection getConnection() {
       return null;
     }
 
@@ -506,11 +505,6 @@ public class TestWALLockup {
     }
 
     @Override
-    public ClusterConnection getClusterConnection() {
-      return null;
-    }
-
-    @Override
     public FileSystem getFileSystem() {
       return null;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index 62ab265..9d3283d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -209,7 +208,7 @@ public class TestReplicationTrackerZKImpl {
     }
 
     @Override
-    public ClusterConnection getConnection() {
+    public Connection getConnection() {
       return null;
     }
 
@@ -245,12 +244,6 @@ public class TestReplicationTrackerZKImpl {
     }
 
     @Override
-    public ClusterConnection getClusterConnection() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
     public FileSystem getFileSystem() {
       return null;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 427f319..3a1320c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -58,7 +58,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -852,8 +851,9 @@ public abstract class TestReplicationSourceManager {
     public CoordinatedStateManager getCoordinatedStateManager() {
       return null;
     }
+
     @Override
-    public ClusterConnection getConnection() {
+    public Connection getConnection() {
       return null;
     }
 
@@ -888,12 +888,6 @@ public abstract class TestReplicationSourceManager {
     }
 
     @Override
-    public ClusterConnection getClusterConnection() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
     public FileSystem getFileSystem() {
       return null;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
index 92c8e54..a2981fb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.coprocessor.HasRegionServerServices;
@@ -211,7 +210,7 @@ public class TestTokenAuthentication {
     }
 
     @Override
-    public ClusterConnection getConnection() {
+    public Connection getConnection() {
       return null;
     }
 
@@ -355,12 +354,6 @@ public class TestTokenAuthentication {
     }
 
     @Override
-    public ClusterConnection getClusterConnection() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index d25ccef..9e29763 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
@@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
@@ -74,9 +73,6 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-
 /**
  * This is the base class for  HBaseFsck's ability to detect reasons for inconsistent tables.
  *
@@ -98,7 +94,7 @@ public class BaseTestHBaseFsck {
   protected static RegionStates regionStates;
   protected static ExecutorService tableExecutorService;
   protected static ScheduledThreadPoolExecutor hbfsckExecutorService;
-  protected static ClusterConnection connection;
+  protected static Connection connection;
   protected static Admin admin;
 
   // for the instance, reset every test run
@@ -298,9 +294,6 @@ public class BaseTestHBaseFsck {
 
   /**
    * delete table in preparation for next test
-   *
-   * @param tablename
-   * @throws IOException
    */
   void cleanupTable(TableName tablename) throws Exception {
     if (tbl != null) {
@@ -319,10 +312,8 @@ public class BaseTestHBaseFsck {
     Collection<ServerName> regionServers = status.getLiveServerMetrics().keySet();
     Map<ServerName, List<String>> mm = new HashMap<>();
     for (ServerName hsi : regionServers) {
-      AdminProtos.AdminService.BlockingInterface server = connection.getAdmin(hsi);
-
       // list all online regions from this region server
-      List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(server);
+      List<RegionInfo> regions = admin.getRegions(hsi);
       List<String> regionNames = new ArrayList<>(regions.size());
       for (RegionInfo hri : regions) {
         regionNames.add(hri.getRegionNameAsString());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
index 13212d2..380c1c7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -104,7 +103,7 @@ public class MockServer implements Server {
   }
 
   @Override
-  public ClusterConnection getConnection() {
+  public Connection getConnection() {
     return null;
   }
 
@@ -115,7 +114,6 @@ public class MockServer implements Server {
 
   @Override
   public boolean isAborted() {
-    // TODO Auto-generated method stub
     return this.aborted;
   }
 
@@ -125,12 +123,6 @@ public class MockServer implements Server {
   }
 
   @Override
-  public ClusterConnection getClusterConnection() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
   public FileSystem getFileSystem() {
     return null;
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
index 0a66ec0..f245384 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
@@ -56,7 +56,7 @@ public abstract class MultiThreadedAction {
 
   protected final TableName tableName;
   protected final Configuration conf;
-  protected final ClusterConnection connection; // all reader / writer threads will share this connection
+  protected final Connection connection; // all reader / writer threads will share this connection
 
   protected int numThreads = 1;
 
@@ -151,7 +151,7 @@ public abstract class MultiThreadedAction {
     this.dataGenerator = dataGen;
     this.tableName = tableName;
     this.actionLetter = actionLetter;
-    this.connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
+    this.connection = ConnectionFactory.createConnection(conf);
   }
 
   public void start(long startKey, long endKey, int numThreads) throws IOException {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
index 0855559..f60ebb6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.io.hfile.TestHFile;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
@@ -74,7 +73,7 @@ public class TestHBaseFsckMOB extends BaseTestHBaseFsck {
         TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
     regionStates = assignmentManager.getRegionStates();
 
-    connection = (ClusterConnection) TEST_UTIL.getConnection();
+    connection = TEST_UTIL.getConnection();
 
     admin = connection.getAdmin();
     admin.setBalancerRunning(false, true);
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java
index 36e513c..abaaba0 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java
@@ -29,9 +29,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ExecutorService;
-
 import javax.net.ssl.SSLException;
-
 import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftConnection.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftConnection.java
index 2c9bf69..7479592 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftConnection.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftConnection.java
@@ -41,11 +41,11 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 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.ConnectionUtils;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
@@ -143,7 +143,7 @@ public class TestThriftConnection {
 
   private static Connection createConnection(int port, boolean useHttp) throws IOException {
     Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
-    conf.set(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL,
+    conf.set(ConnectionUtils.HBASE_CLIENT_CONNECTION_IMPL,
         ThriftConnection.class.getName());
     if (useHttp) {
       conf.set(Constants.HBASE_THRIFT_CLIENT_BUIDLER_CLASS,