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/06/02 03:40:52 UTC

[hbase] 26/27: HBASE-21723 Remove ConnectionImplementation and related classes

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 707d2eee80c83733cf768064c28467225565f3d6
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Fri Apr 26 18:13:22 2019 +0800

    HBASE-21723 Remove ConnectionImplementation and related classes
---
 .../hbase/client/AsyncConnectionConfiguration.java |   12 +-
 .../hbase/client/AsyncMetaRegionLocator.java       |   15 +
 .../hbase/client/AsyncNonMetaRegionLocator.java    |   10 +
 .../apache/hadoop/hbase/client/AsyncProcess.java   |  482 ---
 .../hadoop/hbase/client/AsyncProcessTask.java      |  229 --
 .../hadoop/hbase/client/AsyncRegionLocator.java    |   20 +
 .../hadoop/hbase/client/AsyncRequestFuture.java    |   39 -
 .../hbase/client/AsyncRequestFutureImpl.java       | 1249 -------
 .../apache/hadoop/hbase/client/BatchErrors.java    |   69 -
 .../hadoop/hbase/client/BufferedMutatorImpl.java   |  500 ---
 .../apache/hadoop/hbase/client/Cancellable.java    |   31 -
 .../client/CancellableRegionServerCallable.java    |  123 -
 .../hbase/client/ClientAsyncPrefetchScanner.java   |  193 -
 .../apache/hadoop/hbase/client/ClientScanner.java  |  566 ---
 .../hadoop/hbase/client/ClientServiceCallable.java |   57 -
 .../hadoop/hbase/client/ClientSimpleScanner.java   |   66 -
 .../hbase/client/ConnectionConfiguration.java      |    1 +
 .../hadoop/hbase/client/ConnectionFactory.java     |    8 -
 .../hbase/client/ConnectionImplementation.java     | 2185 -----------
 .../hadoop/hbase/client/ConnectionUtils.java       |   37 -
 .../apache/hadoop/hbase/client/DelayingRunner.java |  116 -
 .../hbase/client/FastFailInterceptorContext.java   |  135 -
 .../hadoop/hbase/client/FlushRegionCallable.java   |   88 -
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java | 3921 --------------------
 .../apache/hadoop/hbase/client/HRegionLocator.java |  117 -
 .../org/apache/hadoop/hbase/client/HTable.java     | 1103 ------
 .../apache/hadoop/hbase/client/MasterCallable.java |  145 -
 .../hbase/client/MasterKeepAliveConnection.java    |   41 -
 .../org/apache/hadoop/hbase/client/MetaCache.java  |  436 ---
 .../apache/hadoop/hbase/client/MultiAction.java    |    3 +-
 .../hadoop/hbase/client/MultiServerCallable.java   |  158 -
 .../client/NoOpRetryableCallerInterceptor.java     |   68 -
 .../client/NoOpRetryingInterceptorContext.java     |   41 -
 .../hbase/client/NoncedRegionServerCallable.java   |   62 -
 .../client/PreemptiveFastFailInterceptor.java      |  350 --
 .../hbase/client/RegionAdminServiceCallable.java   |  195 -
 .../hbase/client/RegionCoprocessorRpcChannel.java  |  104 -
 .../hadoop/hbase/client/RegionServerCallable.java  |  238 --
 .../client/ResultBoundedCompletionService.java     |  273 --
 .../hadoop/hbase/client/RetryingCallable.java      |   73 -
 .../hbase/client/RetryingCallerInterceptor.java    |   98 -
 .../client/RetryingCallerInterceptorContext.java   |   67 -
 .../client/RetryingCallerInterceptorFactory.java   |   82 -
 .../hadoop/hbase/client/RetryingTimeTracker.java   |   60 -
 .../hadoop/hbase/client/ReversedClientScanner.java |   63 -
 .../hbase/client/ReversedScannerCallable.java      |  166 -
 .../hadoop/hbase/client/RpcRetryingCallable.java   |   65 -
 .../hadoop/hbase/client/RpcRetryingCaller.java     |   50 -
 .../hbase/client/RpcRetryingCallerFactory.java     |  130 -
 .../hadoop/hbase/client/RpcRetryingCallerImpl.java |  252 --
 .../client/RpcRetryingCallerWithReadReplicas.java  |  336 --
 .../hadoop/hbase/client/ScannerCallable.java       |  424 ---
 .../hbase/client/ScannerCallableWithReplicas.java  |  444 ---
 .../hbase/client/ShortCircuitMasterConnection.java |  716 ----
 .../hadoop/hbase/client/TestAsyncProcess.java      | 1881 ----------
 .../TestAsyncProcessWithRegionException.java       |  252 --
 .../hadoop/hbase/client/TestClientNoCluster.java   |  820 ----
 .../hadoop/hbase/client/TestClientScanner.java     |  523 ---
 .../hadoop/hbase/client/TestDelayingRunner.java    |   69 -
 .../hadoop/hbase/client/TestProcedureFuture.java   |  188 -
 .../hbase/client/TestReversedScannerCallable.java  |   96 -
 .../hadoop/hbase/client/TestSnapshotFromAdmin.java |  198 -
 .../hbase/test/IntegrationTestBigLinkedList.java   |    3 -
 .../hbase/test/IntegrationTestLoadAndVerify.java   |   39 +-
 ...grationTestWithCellVisibilityLoadAndVerify.java |    3 -
 .../hadoop/hbase/mapred/TableRecordReaderImpl.java |   16 +-
 .../hbase/mapreduce/TableRecordReaderImpl.java     |   12 +-
 .../hadoop/hbase/mob/ExpiredMobFileCleaner.java    |    9 +-
 .../hadoop/hbase/regionserver/HRegionServer.java   |    3 -
 .../regionserver/DumpReplicationQueues.java        |    3 -
 .../hbase/client/HConnectionTestingUtility.java    |  148 +-
 .../hadoop/hbase/client/SleepAtFirstRpcCall.java   |   71 +
 .../org/apache/hadoop/hbase/client/TestAdmin1.java |   19 +-
 .../org/apache/hadoop/hbase/client/TestAdmin2.java |   40 +-
 .../hadoop/hbase/client/TestAsyncAdminBase.java    |    3 +-
 .../hadoop/hbase/client/TestAsyncAdminBuilder.java |    2 +-
 .../hbase/client/TestAsyncClusterAdminApi.java     |    2 +-
 .../hbase/client/TestAsyncClusterAdminApi2.java    |    2 +-
 .../hbase/client/TestAsyncNamespaceAdminApi.java   |    2 +-
 .../hbase/client/TestAsyncProcedureAdminApi.java   |    2 +-
 .../hbase/client/TestAsyncQuotaAdminApi.java       |    2 +-
 .../hbase/client/TestAsyncReplicationAdminApi.java |    2 +-
 .../TestAsyncReplicationAdminApiWithClusters.java  |    2 +-
 .../hadoop/hbase/client/TestCIBadHostname.java     |   36 +-
 .../apache/hadoop/hbase/client/TestCISleep.java    |   60 -
 .../hadoop/hbase/client/TestClientPushback.java    |  109 -
 .../hadoop/hbase/client/TestClientTimeouts.java    |   96 +-
 .../apache/hadoop/hbase/client/TestConnection.java |  356 ++
 .../hbase/client/TestConnectionImplementation.java | 1096 ------
 .../apache/hadoop/hbase/client/TestFastFail.java   |  387 --
 .../hadoop/hbase/client/TestFromClientSide.java    |   30 +-
 .../hbase/client/TestHBaseAdminNoCluster.java      |  336 --
 .../hbase/client/TestIncrementsFromClientSide.java |    4 +-
 .../apache/hadoop/hbase/client/TestMetaCache.java  |  144 +-
 .../client/TestMetaTableAccessorNoCluster.java     |  132 -
 .../hadoop/hbase/client/TestMetaWithReplicas.java  |    8 +-
 .../client/TestMultiActionMetricsFromClient.java   |    7 +-
 .../hadoop/hbase/client/TestMultiParallel.java     |  103 -
 .../hbase/client/TestRegionLocationCaching.java    |    9 +-
 .../hbase/client/TestReplicaWithCluster.java       |  142 -
 .../hadoop/hbase/client/TestReplicasClient.java    |  342 +-
 .../hbase/client/TestResultSizeEstimation.java     |    2 +-
 .../hadoop/hbase/master/TestMasterNoCluster.java   |  194 -
 .../master/assignment/MockMasterServices.java      |    9 +-
 .../regionserver/TestHRegionServerBulkLoad.java    |    4 -
 105 files changed, 717 insertions(+), 23813 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
index 6596578..0fe3868 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
@@ -39,8 +39,6 @@ import static org.apache.hadoop.hbase.HConstants.HBASE_META_SCANNER_CACHING;
 import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_READ_TIMEOUT_KEY;
 import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_TIMEOUT_KEY;
 import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY;
-import static org.apache.hadoop.hbase.client.AsyncProcess.DEFAULT_START_LOG_ERRORS_AFTER_COUNT;
-import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY;
 import static org.apache.hadoop.hbase.client.ConnectionConfiguration.MAX_KEYVALUE_SIZE_DEFAULT;
 import static org.apache.hadoop.hbase.client.ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY;
 import static org.apache.hadoop.hbase.client.ConnectionConfiguration.PRIMARY_CALL_TIMEOUT_MICROSECOND;
@@ -66,6 +64,16 @@ class AsyncConnectionConfiguration {
 
   private static final Logger LOG = LoggerFactory.getLogger(AsyncConnectionConfiguration.class);
 
+  /**
+   * Configure the number of failures after which the client will start logging. A few failures
+   * is fine: region moved, then is not opened, then is overloaded. We try to have an acceptable
+   * heuristic for the number of errors we don't log. 5 was chosen because we wait for 1s at
+   * this stage.
+   */
+  public static final String START_LOG_ERRORS_AFTER_COUNT_KEY =
+      "hbase.client.start.log.errors.counter";
+  public static final int DEFAULT_START_LOG_ERRORS_AFTER_COUNT = 5;
+
   private final long metaOperationTimeoutNs;
 
   // timeout for a whole operation such as get, put or delete. Notice that scan will not be effected
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
index 9cf8bc6..ceb0a3c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 /**
  * The asynchronous locator for meta region.
  */
@@ -133,4 +135,17 @@ class AsyncMetaRegionLocator {
       }
     }
   }
+
+  // only used for testing whether we have cached the location for a region.
+  @VisibleForTesting
+  RegionLocations getRegionLocationInCache() {
+    return metaRegionLocations.get();
+  }
+
+  // only used for testing whether we have cached the location for a table.
+  @VisibleForTesting
+  int getNumberOfCachedRegionLocations() {
+    RegionLocations locs = metaRegionLocations.get();
+    return locs != null ? locs.numNonNullElements() : 0;
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
index 4a9b7dc..5ab023d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
@@ -654,4 +654,14 @@ class AsyncNonMetaRegionLocator {
     }
     return locateRowInCache(tableCache, tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID);
   }
+
+  // only used for testing whether we have cached the location for a table.
+  @VisibleForTesting
+  int getNumberOfCachedRegionLocations(TableName tableName) {
+    TableCache tableCache = cache.get(tableName);
+    if (tableCache == null) {
+      return 0;
+    }
+    return tableCache.cache.values().stream().mapToInt(RegionLocations::numNonNullElements).sum();
+  }
 }
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
deleted file mode 100644
index b0f863f..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ /dev/null
@@ -1,482 +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 org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Consumer;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.client.AsyncProcessTask.SubmittedRows;
-import org.apache.hadoop.hbase.client.RequestController.ReturnCode;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * This class  allows a continuous flow of requests. It's written to be compatible with a
- * synchronous caller such as HTable.
- * <p>
- * The caller sends a buffer of operation, by calling submit. This class extract from this list
- * the operations it can send, i.e. the operations that are on region that are not considered
- * as busy. The process is asynchronous, i.e. it returns immediately when if has finished to
- * iterate on the list. If, and only if, the maximum number of current task is reached, the call
- * to submit will block. Alternatively, the caller can call submitAll, in which case all the
- * operations will be sent. Each call to submit returns a future-like object that can be used
- * to track operation progress.
- * </p>
- * <p>
- * The class manages internally the retries.
- * </p>
- * <p>
- * The errors are tracked inside the Future object that is returned.
- * The results are always tracked inside the Future object and can be retrieved when the call
- * has finished. Partial results can also be retrieved if some part of multi-request failed.
- * </p>
- * <p>
- * This class is thread safe.
- * Internally, the class is thread safe enough to manage simultaneously new submission and results
- * arising from older operations.
- * </p>
- * <p>
- * Internally, this class works with {@link Row}, this mean it could be theoretically used for
- * gets as well.
- * </p>
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-class AsyncProcess {
-  private static final Logger LOG = LoggerFactory.getLogger(AsyncProcess.class);
-  private static final AtomicLong COUNTER = new AtomicLong();
-
-  public static final String PRIMARY_CALL_TIMEOUT_KEY = "hbase.client.primaryCallTimeout.multiget";
-
-  /**
-   * Configure the number of failures after which the client will start logging. A few failures
-   * is fine: region moved, then is not opened, then is overloaded. We try to have an acceptable
-   * heuristic for the number of errors we don't log. 5 was chosen because we wait for 1s at
-   * this stage.
-   */
-  public static final String START_LOG_ERRORS_AFTER_COUNT_KEY =
-      "hbase.client.start.log.errors.counter";
-  public static final int DEFAULT_START_LOG_ERRORS_AFTER_COUNT = 5;
-
-  /**
-   * Configuration to decide whether to log details for batch error
-   */
-  public static final String LOG_DETAILS_FOR_BATCH_ERROR = "hbase.client.log.batcherrors.details";
-
-  /**
-   * Return value from a submit that didn't contain any requests.
-   */
-  private static final AsyncRequestFuture NO_REQS_RESULT = new AsyncRequestFuture() {
-    final Object[] result = new Object[0];
-
-    @Override
-    public boolean hasError() {
-      return false;
-    }
-
-    @Override
-    public RetriesExhaustedWithDetailsException getErrors() {
-      return null;
-    }
-
-    @Override
-    public List<? extends Row> getFailedOperations() {
-      return null;
-    }
-
-    @Override
-    public Object[] getResults() {
-      return result;
-    }
-
-    @Override
-    public void waitUntilDone() throws InterruptedIOException {
-    }
-  };
-
-  // TODO: many of the fields should be made private
-  final long id;
-
-  final ConnectionImplementation connection;
-  private final RpcRetryingCallerFactory rpcCallerFactory;
-  final RpcControllerFactory rpcFactory;
-
-  // Start configuration settings.
-  final int startLogErrorsCnt;
-
-  final long pause;
-  final long pauseForCQTBE;// pause for CallQueueTooBigException, if specified
-  final int numTries;
-  @VisibleForTesting
-  long serverTrackerTimeout;
-  final long primaryCallTimeoutMicroseconds;
-  /** Whether to log details for batch errors */
-  final boolean logBatchErrorDetails;
-  // End configuration settings.
-
-  /**
-   * The traffic control for requests.
-   */
-  @VisibleForTesting
-  final RequestController requestController;
-  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(ConnectionImplementation hc, Configuration conf,
-      RpcRetryingCallerFactory rpcCaller, RpcControllerFactory rpcFactory) {
-    if (hc == null) {
-      throw new IllegalArgumentException("ClusterConnection cannot be null.");
-    }
-
-    this.connection = hc;
-
-    this.id = COUNTER.incrementAndGet();
-
-    this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
-        HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
-    long configuredPauseForCQTBE = conf.getLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, pause);
-    if (configuredPauseForCQTBE < pause) {
-      LOG.warn("The " + HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE + " setting: "
-          + configuredPauseForCQTBE + " is smaller than " + HConstants.HBASE_CLIENT_PAUSE
-          + ", will use " + pause + " instead.");
-      this.pauseForCQTBE = pause;
-    } else {
-      this.pauseForCQTBE = configuredPauseForCQTBE;
-    }
-    // how many times we could try in total, one more than retry number
-    this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
-        HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER) + 1;
-    this.primaryCallTimeoutMicroseconds = conf.getInt(PRIMARY_CALL_TIMEOUT_KEY, 10000);
-    this.startLogErrorsCnt =
-        conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT);
-    this.periodToLog = conf.getInt(LOG_DETAILS_PERIOD, DEFAULT_LOG_DETAILS_PERIOD);
-    // Server tracker allows us to do faster, and yet useful (hopefully), retries.
-    // However, if we are too useful, we might fail very quickly due to retry count limit.
-    // To avoid this, we are going to cheat for now (see HBASE-7659), and calculate maximum
-    // retry time if normal retries were used. Then we will retry until this time runs out.
-    // If we keep hitting one server, the net effect will be the incremental backoff, and
-    // essentially the same number of retries as planned. If we have to do faster retries,
-    // we will do more retries in aggregate, but the user will be none the wiser.
-    this.serverTrackerTimeout = 0L;
-    for (int i = 0; i < this.numTries; ++i) {
-      serverTrackerTimeout = serverTrackerTimeout + ConnectionUtils.getPauseTime(this.pause, i);
-    }
-
-    this.rpcCallerFactory = rpcCaller;
-    this.rpcFactory = rpcFactory;
-    this.logBatchErrorDetails = conf.getBoolean(LOG_DETAILS_FOR_BATCH_ERROR, false);
-
-    this.requestController = RequestControllerFactory.create(conf);
-  }
-
-  /**
-   * The submitted task may be not accomplished at all if there are too many running tasks or
-   * other limits.
-   * @param <CResult> The class to cast the result
-   * @param task The setting and data
-   * @return AsyncRequestFuture
-   */
-  public <CResult> AsyncRequestFuture submit(AsyncProcessTask<CResult> task) throws InterruptedIOException {
-    AsyncRequestFuture reqFuture = checkTask(task);
-    if (reqFuture != null) {
-      return reqFuture;
-    }
-    SubmittedRows submittedRows = task.getSubmittedRows() == null ? SubmittedRows.ALL : task.getSubmittedRows();
-    switch (submittedRows) {
-      case ALL:
-        return submitAll(task);
-      case AT_LEAST_ONE:
-        return submit(task, true);
-      default:
-        return submit(task, false);
-    }
-  }
-
-  /**
-   * Extract from the rows list what we can submit. The rows we can not submit are kept in the
-   * list. Does not send requests to replicas (not currently used for anything other
-   * than streaming puts anyway).
-   *
-   * @param task The setting and data
-   * @param atLeastOne true if we should submit at least a subset.
-   */
-  private <CResult> AsyncRequestFuture submit(AsyncProcessTask<CResult> task,
-    boolean atLeastOne) throws InterruptedIOException {
-    TableName tableName = task.getTableName();
-    RowAccess<? extends Row> rows = task.getRowAccess();
-    Map<ServerName, MultiAction> actionsByServer = new HashMap<>();
-    List<Action> retainedActions = new ArrayList<>(rows.size());
-
-    NonceGenerator ng = this.connection.getNonceGenerator();
-    long nonceGroup = ng.getNonceGroup(); // Currently, nonce group is per entire client.
-
-    // Location errors that happen before we decide what requests to take.
-    List<Exception> locationErrors = null;
-    List<Integer> locationErrorRows = null;
-    RequestController.Checker checker = requestController.newChecker();
-    boolean firstIter = true;
-    do {
-      // Wait until there is at least one slot for a new task.
-      requestController.waitForFreeSlot(id, periodToLog, getLogger(tableName, -1));
-      int posInList = -1;
-      if (!firstIter) {
-        checker.reset();
-      }
-      Iterator<? extends Row> it = rows.iterator();
-      while (it.hasNext()) {
-        Row r = it.next();
-        HRegionLocation loc;
-        try {
-          if (r == null) {
-            throw new IllegalArgumentException("#" + id + ", row cannot be null");
-          }
-          // Make sure we get 0-s replica.
-          RegionLocations locs = connection.locateRegion(
-              tableName, r.getRow(), true, true, RegionReplicaUtil.DEFAULT_REPLICA_ID);
-          if (locs == null || locs.isEmpty() || locs.getDefaultRegionLocation() == null) {
-            throw new IOException("#" + id + ", no location found, aborting submit for"
-                + " tableName=" + tableName + " rowkey=" + Bytes.toStringBinary(r.getRow()));
-          }
-          loc = locs.getDefaultRegionLocation();
-        } catch (IOException ex) {
-          locationErrors = new ArrayList<>(1);
-          locationErrorRows = new ArrayList<>(1);
-          LOG.error("Failed to get region location ", ex);
-          // This action failed before creating ars. Retain it, but do not add to submit list.
-          // We will then add it to ars in an already-failed state.
-
-          int priority = HConstants.NORMAL_QOS;
-          if (r instanceof Mutation) {
-            priority = ((Mutation) r).getPriority();
-          }
-          retainedActions.add(new Action(r, ++posInList, priority));
-          locationErrors.add(ex);
-          locationErrorRows.add(posInList);
-          it.remove();
-          break; // Backward compat: we stop considering actions on location error.
-        }
-        ReturnCode code = checker.canTakeRow(loc, r);
-        if (code == ReturnCode.END) {
-          break;
-        }
-        if (code == ReturnCode.INCLUDE) {
-          int priority = HConstants.NORMAL_QOS;
-          if (r instanceof Mutation) {
-            priority = ((Mutation) r).getPriority();
-          }
-          Action action = new Action(r, ++posInList, priority);
-          setNonce(ng, r, action);
-          retainedActions.add(action);
-          // TODO: replica-get is not supported on this path
-          byte[] regionName = loc.getRegionInfo().getRegionName();
-          addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup);
-          it.remove();
-        }
-      }
-      firstIter = false;
-    } while (retainedActions.isEmpty() && atLeastOne && (locationErrors == null));
-
-    if (retainedActions.isEmpty()) return NO_REQS_RESULT;
-
-    return submitMultiActions(task, retainedActions, nonceGroup,
-        locationErrors, locationErrorRows, actionsByServer);
-  }
-
-  <CResult> AsyncRequestFuture submitMultiActions(AsyncProcessTask task,
-      List<Action> retainedActions, long nonceGroup, List<Exception> locationErrors,
-      List<Integer> locationErrorRows, Map<ServerName, MultiAction> actionsByServer) {
-    AsyncRequestFutureImpl<CResult> ars = createAsyncRequestFuture(task, retainedActions, nonceGroup);
-    // Add location errors if any
-    if (locationErrors != null) {
-      for (int i = 0; i < locationErrors.size(); ++i) {
-        int originalIndex = locationErrorRows.get(i);
-        Row row = retainedActions.get(originalIndex).getAction();
-        ars.manageError(originalIndex, row,
-            AsyncRequestFutureImpl.Retry.NO_LOCATION_PROBLEM, locationErrors.get(i), null);
-      }
-    }
-    ars.sendMultiAction(actionsByServer, 1, null, false);
-    return ars;
-  }
-
-  /**
-   * Helper that is used when grouping the actions per region server.
-   *
-   * @param server - server
-   * @param regionName - regionName
-   * @param action - the action to add to the multiaction
-   * @param actionsByServer the multiaction per server
-   * @param nonceGroup Nonce group.
-   */
-  static void addAction(ServerName server, byte[] regionName, Action action,
-      Map<ServerName, MultiAction> actionsByServer, long nonceGroup) {
-    MultiAction multiAction = actionsByServer.get(server);
-    if (multiAction == null) {
-      multiAction = new MultiAction();
-      actionsByServer.put(server, multiAction);
-    }
-    if (action.hasNonce() && !multiAction.hasNonceGroup()) {
-      multiAction.setNonceGroup(nonceGroup);
-    }
-
-    multiAction.add(regionName, action);
-  }
-
-  /**
-   * Submit immediately the list of rows, whatever the server status. Kept for backward
-   * compatibility: it allows to be used with the batch interface that return an array of objects.
-   * @param task The setting and data
-   */
-  private <CResult> AsyncRequestFuture submitAll(AsyncProcessTask task) {
-    RowAccess<? extends Row> rows = task.getRowAccess();
-    List<Action> actions = new ArrayList<>(rows.size());
-
-    // The position will be used by the processBatch to match the object array returned.
-    int posInList = -1;
-    NonceGenerator ng = this.connection.getNonceGenerator();
-    int highestPriority = HConstants.PRIORITY_UNSET;
-    for (Row r : rows) {
-      posInList++;
-      if (r instanceof Put) {
-        Put put = (Put) r;
-        if (put.isEmpty()) {
-          throw new IllegalArgumentException("No columns to insert for #" + (posInList+1)+ " item");
-        }
-        highestPriority = Math.max(put.getPriority(), highestPriority);
-      }
-      Action action = new Action(r, posInList, highestPriority);
-      setNonce(ng, r, action);
-      actions.add(action);
-    }
-    AsyncRequestFutureImpl<CResult> ars = createAsyncRequestFuture(task, actions, ng.getNonceGroup());
-    ars.groupAndSendMultiAction(actions, 1);
-    return ars;
-  }
-
-  private <CResult> AsyncRequestFuture checkTask(AsyncProcessTask<CResult> task) {
-    if (task.getRowAccess() == null || task.getRowAccess().isEmpty()) {
-      return NO_REQS_RESULT;
-    }
-    Objects.requireNonNull(task.getPool(), "The pool can't be NULL");
-    checkOperationTimeout(task.getOperationTimeout());
-    checkRpcTimeout(task.getRpcTimeout());
-    return null;
-  }
-
-  private void setNonce(NonceGenerator ng, Row r, Action action) {
-    if (!(r instanceof Append) && !(r instanceof Increment)) return;
-    action.setNonce(ng.newNonce()); // Action handles NO_NONCE, so it's ok if ng is disabled.
-  }
-
-  private int checkTimeout(String name, int timeout) {
-    if (timeout < 0) {
-      throw new RuntimeException("The " + name + " must be bigger than zero,"
-        + "current value is" + timeout);
-    }
-    return timeout;
-  }
-  private int checkOperationTimeout(int operationTimeout) {
-    return checkTimeout("operation timeout", operationTimeout);
-  }
-
-  private int checkRpcTimeout(int rpcTimeout) {
-    return checkTimeout("rpc timeout", rpcTimeout);
-  }
-
-  @VisibleForTesting
-  <CResult> AsyncRequestFutureImpl<CResult> createAsyncRequestFuture(
-      AsyncProcessTask task, List<Action> actions, long nonceGroup) {
-    return new AsyncRequestFutureImpl<>(task, actions, nonceGroup, this);
-  }
-
-  /** Wait until the async does not have more than max tasks in progress. */
-  protected void waitForMaximumCurrentTasks(int max, TableName tableName)
-      throws InterruptedIOException {
-    requestController.waitForMaximumCurrentTasks(max, id, periodToLog,
-      getLogger(tableName, max));
-  }
-
-  private Consumer<Long> getLogger(TableName tableName, long max) {
-    return (currentInProgress) -> {
-      LOG.info("#" + id + (max < 0 ?
-          ", waiting for any free slot" :
-          ", waiting for some tasks to finish. Expected max=" + max) + ", tasksInProgress="
-          + currentInProgress + (tableName == null ? "" : ", tableName=" + tableName));
-    };
-  }
-
-  void incTaskCounters(Collection<byte[]> regions, ServerName sn) {
-    requestController.incTaskCounters(regions, sn);
-  }
-
-
-  void decTaskCounters(Collection<byte[]> regions, ServerName sn) {
-    requestController.decTaskCounters(regions, sn);
-  }
-
-  /**
-   * Create a caller. Isolated to be easily overridden in the tests.
-   */
-  @VisibleForTesting
-  protected RpcRetryingCaller<AbstractResponse> createCaller(
-      CancellableRegionServerCallable callable, int rpcTimeout) {
-    return rpcCallerFactory.<AbstractResponse> newCaller(checkRpcTimeout(rpcTimeout));
-  }
-
-
-  /**
-   * Creates the server error tracker to use inside process.
-   * Currently, to preserve the main assumption about current retries, and to work well with
-   * the retry-limit-based calculation, the calculation is local per Process object.
-   * We may benefit from connection-wide tracking of server errors.
-   * @return ServerErrorTracker to use, null if there is no ServerErrorTracker on this connection
-   */
-  ConnectionImplementation.ServerErrorTracker createServerErrorTracker() {
-    return new ConnectionImplementation.ServerErrorTracker(
-        this.serverTrackerTimeout, this.numTries);
-  }
-
-  static boolean isReplicaGet(Row row) {
-    return (row instanceof Get) && (((Get)row).getConsistency() == Consistency.TIMELINE);
-  }
-
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java
deleted file mode 100644
index 5a2bbfe..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java
+++ /dev/null
@@ -1,229 +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.util.Iterator;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.client.coprocessor.Batch;
-
-/**
- * Contains the attributes of a task which will be executed
- * by {@link org.apache.hadoop.hbase.client.AsyncProcess}.
- * The attributes will be validated by AsyncProcess.
- * It's intended for advanced client applications.
- * @param <T> The type of response from server-side
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class AsyncProcessTask<T> {
-  /**
-   * The number of processed rows.
-   * The AsyncProcess has traffic control which may reject some rows.
-   */
-  public enum SubmittedRows {
-    ALL,
-    AT_LEAST_ONE,
-    NORMAL
-  }
-  public static <T> Builder<T> newBuilder(final Batch.Callback<T> callback) {
-    return new Builder<>(callback);
-  }
-  public static Builder newBuilder() {
-    return new Builder();
-  }
-
-  public static class Builder<T> {
-
-    private ExecutorService pool;
-    private TableName tableName;
-    private RowAccess<? extends Row> rows;
-    private SubmittedRows submittedRows = SubmittedRows.ALL;
-    private Batch.Callback<T> callback;
-    private boolean needResults;
-    private int rpcTimeout;
-    private int operationTimeout;
-    private CancellableRegionServerCallable callable;
-    private Object[] results;
-
-    private Builder() {
-    }
-
-    private Builder(Batch.Callback<T> callback) {
-      this.callback = callback;
-    }
-
-    Builder<T> setResults(Object[] results) {
-      this.results = results;
-      if (results != null && results.length != 0) {
-        setNeedResults(true);
-      }
-      return this;
-    }
-
-    public Builder<T> setPool(ExecutorService pool) {
-      this.pool = pool;
-      return this;
-    }
-
-    public Builder<T> setRpcTimeout(int rpcTimeout) {
-      this.rpcTimeout = rpcTimeout;
-      return this;
-    }
-
-    public Builder<T> setOperationTimeout(int operationTimeout) {
-      this.operationTimeout = operationTimeout;
-      return this;
-    }
-
-    public Builder<T> setTableName(TableName tableName) {
-      this.tableName = tableName;
-      return this;
-    }
-
-    public Builder<T> setRowAccess(List<? extends Row> rows) {
-      this.rows = new ListRowAccess<>(rows);
-      return this;
-    }
-
-    public Builder<T> setRowAccess(RowAccess<? extends Row> rows) {
-      this.rows = rows;
-      return this;
-    }
-
-    public Builder<T> setSubmittedRows(SubmittedRows submittedRows) {
-      this.submittedRows = submittedRows;
-      return this;
-    }
-
-    public Builder<T> setNeedResults(boolean needResults) {
-      this.needResults = needResults;
-      return this;
-    }
-
-    Builder<T> setCallable(CancellableRegionServerCallable callable) {
-      this.callable = callable;
-      return this;
-    }
-
-    public AsyncProcessTask<T> build() {
-      return new AsyncProcessTask<>(pool, tableName, rows, submittedRows,
-              callback, callable, needResults, rpcTimeout, operationTimeout, results);
-    }
-  }
-  private final ExecutorService pool;
-  private final TableName tableName;
-  private final RowAccess<? extends Row> rows;
-  private final SubmittedRows submittedRows;
-  private final Batch.Callback<T> callback;
-  private final CancellableRegionServerCallable callable;
-  private final boolean needResults;
-  private final int rpcTimeout;
-  private final int operationTimeout;
-  private final Object[] results;
-  AsyncProcessTask(AsyncProcessTask<T> task) {
-    this(task.getPool(), task.getTableName(), task.getRowAccess(),
-        task.getSubmittedRows(), task.getCallback(), task.getCallable(),
-        task.getNeedResults(), task.getRpcTimeout(), task.getOperationTimeout(),
-        task.getResults());
-  }
-  AsyncProcessTask(ExecutorService pool, TableName tableName,
-          RowAccess<? extends Row> rows, SubmittedRows size, Batch.Callback<T> callback,
-          CancellableRegionServerCallable callable, boolean needResults,
-          int rpcTimeout, int operationTimeout, Object[] results) {
-    this.pool = pool;
-    this.tableName = tableName;
-    this.rows = rows;
-    this.submittedRows = size;
-    this.callback = callback;
-    this.callable = callable;
-    this.needResults = needResults;
-    this.rpcTimeout = rpcTimeout;
-    this.operationTimeout = operationTimeout;
-    this.results = results;
-  }
-
-  public int getOperationTimeout() {
-    return operationTimeout;
-  }
-
-  public ExecutorService getPool() {
-    return pool;
-  }
-
-  public TableName getTableName() {
-    return tableName;
-  }
-
-  public RowAccess<? extends Row> getRowAccess() {
-    return rows;
-  }
-
-  public SubmittedRows getSubmittedRows() {
-    return submittedRows;
-  }
-
-  public Batch.Callback<T> getCallback() {
-    return callback;
-  }
-
-  CancellableRegionServerCallable getCallable() {
-    return callable;
-  }
-
-  Object[] getResults() {
-    return results;
-  }
-
-  public boolean getNeedResults() {
-    return needResults;
-  }
-
-  public int getRpcTimeout() {
-    return rpcTimeout;
-  }
-
-  static class ListRowAccess<T> implements RowAccess<T> {
-
-    private final List<T> data;
-
-    ListRowAccess(final List<T> data) {
-      this.data = data;
-    }
-
-    @Override
-    public int size() {
-      return data.size();
-    }
-
-    @Override
-    public boolean isEmpty() {
-      return data.isEmpty();
-    }
-
-    @Override
-    public Iterator<T> iterator() {
-      return data.iterator();
-    }
-  }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
index 9e1d5e8..09eabfc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
@@ -179,4 +179,24 @@ class AsyncRegionLocator {
   AsyncNonMetaRegionLocator getNonMetaRegionLocator() {
     return nonMetaRegionLocator;
   }
+
+  // only used for testing whether we have cached the location for a region.
+  @VisibleForTesting
+  RegionLocations getRegionLocationInCache(TableName tableName, byte[] row) {
+    if (TableName.isMetaTableName(tableName)) {
+      return metaRegionLocator.getRegionLocationInCache();
+    } else {
+      return nonMetaRegionLocator.getRegionLocationInCache(tableName, row);
+    }
+  }
+
+  // only used for testing whether we have cached the location for a table.
+  @VisibleForTesting
+  int getNumberOfCachedRegionLocations(TableName tableName) {
+    if (TableName.isMetaTableName(tableName)) {
+      return metaRegionLocator.getNumberOfCachedRegionLocations();
+    } else {
+      return nonMetaRegionLocator.getNumberOfCachedRegionLocations(tableName);
+    }
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java
deleted file mode 100644
index b91e094..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java
+++ /dev/null
@@ -1,39 +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 org.apache.yetus.audience.InterfaceAudience;
-
-import java.io.InterruptedIOException;
-import java.util.List;
-
-/**
- * The context used to wait for results from one submit call. If submit call is made with
- * needResults false, results will not be saved.
- * @since 2.0.0
- */
-@InterfaceAudience.Private
-public interface AsyncRequestFuture {
-  public boolean hasError();
-  public RetriesExhaustedWithDetailsException getErrors();
-  public List<? extends Row> getFailedOperations();
-  public Object[] getResults() throws InterruptedIOException;
-  /** Wait until all tasks are executed, successfully or not. */
-  public void waitUntilDone() throws InterruptedIOException;
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
deleted file mode 100644
index e46a50e..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
+++ /dev/null
@@ -1,1249 +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.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.hadoop.hbase.CallQueueTooBigException;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.RetryImmediatelyException;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
-import org.apache.hadoop.hbase.client.coprocessor.Batch;
-import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
-import org.apache.hadoop.hbase.trace.TraceUtil;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.htrace.core.Tracer;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
-/**
- * The context, and return value, for a single submit/submitAll call.
- * Note on how this class (one AP submit) works. Initially, all requests are split into groups
- * by server; request is sent to each server in parallel; the RPC calls are not async so a
- * thread per server is used. Every time some actions fail, regions/locations might have
- * changed, so we re-group them by server and region again and send these groups in parallel
- * too. The result, in case of retries, is a "tree" of threads, with parent exiting after
- * scheduling children. This is why lots of code doesn't require any synchronization.
- */
-@InterfaceAudience.Private
-class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
-
-  private static final Logger LOG = LoggerFactory.getLogger(AsyncRequestFutureImpl.class);
-
-  private RetryingTimeTracker tracker;
-
-  /**
-   * Runnable (that can be submitted to thread pool) that waits for when it's time
-   * to issue replica calls, finds region replicas, groups the requests by replica and
-   * issues the calls (on separate threads, via sendMultiAction).
-   * This is done on a separate thread because we don't want to wait on user thread for
-   * our asynchronous call, and usually we have to wait before making replica calls.
-   */
-  private final class ReplicaCallIssuingRunnable implements Runnable {
-    private final long startTime;
-    private final List<Action> initialActions;
-
-    public ReplicaCallIssuingRunnable(List<Action> initialActions, long startTime) {
-      this.initialActions = initialActions;
-      this.startTime = startTime;
-    }
-
-    @Override
-    public void run() {
-      boolean done = false;
-      if (asyncProcess.primaryCallTimeoutMicroseconds > 0) {
-        try {
-          done = waitUntilDone(startTime * 1000L + asyncProcess.primaryCallTimeoutMicroseconds);
-        } catch (InterruptedException ex) {
-          LOG.error("Replica thread interrupted - no replica calls {}", ex.getMessage());
-          return;
-        }
-      }
-      if (done) return; // Done within primary timeout
-      Map<ServerName, MultiAction> actionsByServer = new HashMap<>();
-      List<Action> unknownLocActions = new ArrayList<>();
-      if (replicaGetIndices == null) {
-        for (int i = 0; i < results.length; ++i) {
-          addReplicaActions(i, actionsByServer, unknownLocActions);
-        }
-      } else {
-        for (int replicaGetIndice : replicaGetIndices) {
-          addReplicaActions(replicaGetIndice, actionsByServer, unknownLocActions);
-        }
-      }
-      if (!actionsByServer.isEmpty()) {
-        sendMultiAction(actionsByServer, 1, null, unknownLocActions.isEmpty());
-      }
-      if (!unknownLocActions.isEmpty()) {
-        actionsByServer = new HashMap<>();
-        for (Action action : unknownLocActions) {
-          addReplicaActionsAgain(action, actionsByServer);
-        }
-        // Some actions may have completely failed, they are handled inside addAgain.
-        if (!actionsByServer.isEmpty()) {
-          sendMultiAction(actionsByServer, 1, null, true);
-        }
-      }
-    }
-
-    /**
-     * Add replica actions to action map by server.
-     * @param index Index of the original action.
-     * @param actionsByServer The map by server to add it to.
-     */
-    private void addReplicaActions(int index, Map<ServerName, MultiAction> actionsByServer,
-                                   List<Action> unknownReplicaActions) {
-      if (results[index] != null) return; // opportunistic. Never goes from non-null to null.
-      Action action = initialActions.get(index);
-      RegionLocations loc = findAllLocationsOrFail(action, true);
-      if (loc == null) return;
-      HRegionLocation[] locs = loc.getRegionLocations();
-      if (locs.length == 1) {
-        LOG.warn("No replicas found for {}", action.getAction());
-        return;
-      }
-      synchronized (replicaResultLock) {
-        // Don't run replica calls if the original has finished. We could do it e.g. if
-        // original has already failed before first replica call (unlikely given retries),
-        // but that would require additional synchronization w.r.t. returning to caller.
-        if (results[index] != null) return;
-        // We set the number of calls here. After that any path must call setResult/setError.
-        // True even for replicas that are not found - if we refuse to send we MUST set error.
-        updateResult(index, new ReplicaResultState(locs.length));
-      }
-      for (int i = 1; i < locs.length; ++i) {
-        Action replicaAction = new Action(action, i);
-        if (locs[i] != null) {
-          asyncProcess.addAction(locs[i].getServerName(), locs[i].getRegionInfo().getRegionName(),
-              replicaAction, actionsByServer, nonceGroup);
-        } else {
-          unknownReplicaActions.add(replicaAction);
-        }
-      }
-    }
-
-    private void addReplicaActionsAgain(
-        Action action, Map<ServerName, MultiAction> actionsByServer) {
-      if (action.getReplicaId() == RegionReplicaUtil.DEFAULT_REPLICA_ID) {
-        throw new AssertionError("Cannot have default replica here");
-      }
-      HRegionLocation loc = getReplicaLocationOrFail(action);
-      if (loc == null) return;
-      asyncProcess.addAction(loc.getServerName(), loc.getRegionInfo().getRegionName(),
-          action, actionsByServer, nonceGroup);
-    }
-  }
-
-  /**
-   * Runnable (that can be submitted to thread pool) that submits MultiAction to a
-   * single server. The server call is synchronous, therefore we do it on a thread pool.
-   */
-  @VisibleForTesting
-  final class SingleServerRequestRunnable implements Runnable {
-    private final MultiAction multiAction;
-    private final int numAttempt;
-    private final ServerName server;
-    private final Set<CancellableRegionServerCallable> callsInProgress;
-    @VisibleForTesting
-    SingleServerRequestRunnable(
-        MultiAction multiAction, int numAttempt, ServerName server,
-        Set<CancellableRegionServerCallable> callsInProgress) {
-      this.multiAction = multiAction;
-      this.numAttempt = numAttempt;
-      this.server = server;
-      this.callsInProgress = callsInProgress;
-    }
-
-    @Override
-    public void run() {
-      AbstractResponse res = null;
-      CancellableRegionServerCallable callable = currentCallable;
-      try {
-        // setup the callable based on the actions, if we don't have one already from the request
-        if (callable == null) {
-          callable = createCallable(server, tableName, multiAction);
-        }
-        RpcRetryingCaller<AbstractResponse> caller = asyncProcess.createCaller(callable,rpcTimeout);
-        try {
-          if (callsInProgress != null) {
-            callsInProgress.add(callable);
-          }
-          res = caller.callWithoutRetries(callable, operationTimeout);
-          if (res == null) {
-            // Cancelled
-            return;
-          }
-        } catch (IOException e) {
-          // The service itself failed . It may be an error coming from the communication
-          //   layer, but, as well, a functional error raised by the server.
-          receiveGlobalFailure(multiAction, server, numAttempt, e);
-          return;
-        } catch (Throwable t) {
-          // This should not happen. Let's log & retry anyway.
-          LOG.error("id=" + asyncProcess.id + ", caught throwable. Unexpected." +
-              " Retrying. Server=" + server + ", tableName=" + tableName, t);
-          receiveGlobalFailure(multiAction, server, numAttempt, t);
-          return;
-        }
-        if (res.type() == AbstractResponse.ResponseType.MULTI) {
-          // Normal case: we received an answer from the server, and it's not an exception.
-          receiveMultiAction(multiAction, server, (MultiResponse) res, numAttempt);
-        } else {
-          if (results != null) {
-            SingleResponse singleResponse = (SingleResponse) res;
-            updateResult(0, singleResponse.getEntry());
-          }
-          decActionCounter(1);
-        }
-      } catch (Throwable t) {
-        // Something really bad happened. We are on the send thread that will now die.
-        LOG.error("id=" + asyncProcess.id + " error for " + tableName + " processing " + server, t);
-        throw new RuntimeException(t);
-      } finally {
-        asyncProcess.decTaskCounters(multiAction.getRegions(), server);
-        if (callsInProgress != null && callable != null && res != null) {
-          callsInProgress.remove(callable);
-        }
-      }
-    }
-  }
-
-  private final Batch.Callback<CResult> callback;
-  private final BatchErrors errors;
-  private final ConnectionImplementation.ServerErrorTracker errorsByServer;
-  private final ExecutorService pool;
-  private final Set<CancellableRegionServerCallable> callsInProgress;
-
-
-  private final TableName tableName;
-  private final AtomicLong actionsInProgress = new AtomicLong(-1);
-  /**
-   * The lock controls access to results. It is only held when populating results where
-   * there might be several callers (eventual consistency gets). For other requests,
-   * there's one unique call going on per result index.
-   */
-  private final Object replicaResultLock = new Object();
-  /**
-   * Result array.  Null if results are not needed. Otherwise, each index corresponds to
-   * the action index in initial actions submitted. For most request types, has null-s for
-   * requests that are not done, and result/exception for those that are done.
-   * For eventual-consistency gets, initially the same applies; at some point, replica calls
-   * might be started, and ReplicaResultState is put at the corresponding indices. The
-   * returning calls check the type to detect when this is the case. After all calls are done,
-   * ReplicaResultState-s are replaced with results for the user.
-   */
-  private final Object[] results;
-  /**
-   * Indices of replica gets in results. If null, all or no actions are replica-gets.
-   */
-  private final int[] replicaGetIndices;
-  private final boolean hasAnyReplicaGets;
-  private final long nonceGroup;
-  private final CancellableRegionServerCallable currentCallable;
-  private final int operationTimeout;
-  private final int rpcTimeout;
-  private final AsyncProcess asyncProcess;
-
-  /**
-   * For {@link AsyncRequestFutureImpl#manageError(int, Row, Retry, Throwable, ServerName)}. Only
-   * used to make logging more clear, we don't actually care why we don't retry.
-   */
-  public enum Retry {
-    YES,
-    NO_LOCATION_PROBLEM,
-    NO_NOT_RETRIABLE,
-    NO_RETRIES_EXHAUSTED,
-    NO_OTHER_SUCCEEDED
-  }
-
-  /** Sync point for calls to multiple replicas for the same user request (Get).
-   * Created and put in the results array (we assume replica calls require results) when
-   * the replica calls are launched. See results for details of this process.
-   * POJO, all fields are public. To modify them, the object itself is locked. */
-  private static class ReplicaResultState {
-    public ReplicaResultState(int callCount) {
-      this.callCount = callCount;
-    }
-
-    /** Number of calls outstanding, or 0 if a call succeeded (even with others outstanding). */
-    int callCount;
-    /** Errors for which it is not decided whether we will report them to user. If one of the
-     * calls succeeds, we will discard the errors that may have happened in the other calls. */
-    BatchErrors replicaErrors = null;
-
-    @Override
-    public String toString() {
-      return "[call count " + callCount + "; errors " + replicaErrors + "]";
-    }
-  }
-
-  public AsyncRequestFutureImpl(AsyncProcessTask task, List<Action> actions,
-      long nonceGroup, AsyncProcess asyncProcess) {
-    this.pool = task.getPool();
-    this.callback = task.getCallback();
-    this.nonceGroup = nonceGroup;
-    this.tableName = task.getTableName();
-    this.actionsInProgress.set(actions.size());
-    if (task.getResults() == null) {
-      results = task.getNeedResults() ? new Object[actions.size()] : null;
-    } else {
-      if (task.getResults().length != actions.size()) {
-        throw new AssertionError("results.length");
-      }
-      this.results = task.getResults();
-      for (int i = 0; i != this.results.length; ++i) {
-        results[i] = null;
-      }
-    }
-    List<Integer> replicaGetIndices = null;
-    boolean hasAnyReplicaGets = false;
-    if (results != null) {
-      // Check to see if any requests might require replica calls.
-      // We expect that many requests will consist of all or no multi-replica gets; in such
-      // cases we would just use a boolean (hasAnyReplicaGets). If there's a mix, we will
-      // store the list of action indexes for which replica gets are possible, and set
-      // hasAnyReplicaGets to true.
-      boolean hasAnyNonReplicaReqs = false;
-      int posInList = 0;
-      for (Action action : actions) {
-        boolean isReplicaGet = AsyncProcess.isReplicaGet(action.getAction());
-        if (isReplicaGet) {
-          hasAnyReplicaGets = true;
-          if (hasAnyNonReplicaReqs) { // Mixed case
-            if (replicaGetIndices == null) {
-              replicaGetIndices = new ArrayList<>(actions.size() - 1);
-            }
-            replicaGetIndices.add(posInList);
-          }
-        } else if (!hasAnyNonReplicaReqs) {
-          // The first non-multi-replica request in the action list.
-          hasAnyNonReplicaReqs = true;
-          if (posInList > 0) {
-            // Add all the previous requests to the index lists. We know they are all
-            // replica-gets because this is the first non-multi-replica request in the list.
-            replicaGetIndices = new ArrayList<>(actions.size() - 1);
-            for (int i = 0; i < posInList; ++i) {
-              replicaGetIndices.add(i);
-            }
-          }
-        }
-        ++posInList;
-      }
-    }
-    this.hasAnyReplicaGets = hasAnyReplicaGets;
-    if (replicaGetIndices != null) {
-      this.replicaGetIndices = new int[replicaGetIndices.size()];
-      int i = 0;
-      for (Integer el : replicaGetIndices) {
-        this.replicaGetIndices[i++] = el;
-      }
-    } else {
-      this.replicaGetIndices = null;
-    }
-    this.callsInProgress = !hasAnyReplicaGets ? null :
-        Collections.newSetFromMap(
-            new ConcurrentHashMap<CancellableRegionServerCallable, Boolean>());
-    this.asyncProcess = asyncProcess;
-    this.errorsByServer = createServerErrorTracker();
-    this.errors = new BatchErrors();
-    this.operationTimeout = task.getOperationTimeout();
-    this.rpcTimeout = task.getRpcTimeout();
-    this.currentCallable = task.getCallable();
-    if (task.getCallable() == null) {
-      tracker = new RetryingTimeTracker().start();
-    }
-  }
-
-  @VisibleForTesting
-  protected Set<CancellableRegionServerCallable> getCallsInProgress() {
-    return callsInProgress;
-  }
-
-  @VisibleForTesting
-  SingleServerRequestRunnable createSingleServerRequest(MultiAction multiAction, int numAttempt, ServerName server,
-        Set<CancellableRegionServerCallable> callsInProgress) {
-    return new SingleServerRequestRunnable(multiAction, numAttempt, server, callsInProgress);
-  }
-
-  /**
-   * Group a list of actions per region servers, and send them.
-   *
-   * @param currentActions - the list of row to submit
-   * @param numAttempt - the current numAttempt (first attempt is 1)
-   */
-  void groupAndSendMultiAction(List<Action> currentActions, int numAttempt) {
-    Map<ServerName, MultiAction> actionsByServer = new HashMap<>();
-
-    boolean isReplica = false;
-    List<Action> unknownReplicaActions = null;
-    for (Action action : currentActions) {
-      RegionLocations locs = findAllLocationsOrFail(action, true);
-      if (locs == null) continue;
-      boolean isReplicaAction = !RegionReplicaUtil.isDefaultReplica(action.getReplicaId());
-      if (isReplica && !isReplicaAction) {
-        // This is the property of the current implementation, not a requirement.
-        throw new AssertionError("Replica and non-replica actions in the same retry");
-      }
-      isReplica = isReplicaAction;
-      HRegionLocation loc = locs.getRegionLocation(action.getReplicaId());
-      if (loc == null || loc.getServerName() == null) {
-        if (isReplica) {
-          if (unknownReplicaActions == null) {
-            unknownReplicaActions = new ArrayList<>(1);
-          }
-          unknownReplicaActions.add(action);
-        } else {
-          // TODO: relies on primary location always being fetched
-          manageLocationError(action, null);
-        }
-      } else {
-        byte[] regionName = loc.getRegionInfo().getRegionName();
-        AsyncProcess.addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup);
-      }
-    }
-    boolean doStartReplica = (numAttempt == 1 && !isReplica && hasAnyReplicaGets);
-    boolean hasUnknown = unknownReplicaActions != null && !unknownReplicaActions.isEmpty();
-
-    if (!actionsByServer.isEmpty()) {
-      // If this is a first attempt to group and send, no replicas, we need replica thread.
-      sendMultiAction(actionsByServer, numAttempt, (doStartReplica && !hasUnknown)
-          ? currentActions : null, numAttempt > 1 && !hasUnknown);
-    }
-
-    if (hasUnknown) {
-      actionsByServer = new HashMap<>();
-      for (Action action : unknownReplicaActions) {
-        HRegionLocation loc = getReplicaLocationOrFail(action);
-        if (loc == null) continue;
-        byte[] regionName = loc.getRegionInfo().getRegionName();
-        AsyncProcess.addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup);
-      }
-      if (!actionsByServer.isEmpty()) {
-        sendMultiAction(
-            actionsByServer, numAttempt, doStartReplica ? currentActions : null, true);
-      }
-    }
-  }
-
-  private HRegionLocation getReplicaLocationOrFail(Action action) {
-    // We are going to try get location once again. For each action, we'll do it once
-    // from cache, because the previous calls in the loop might populate it.
-    int replicaId = action.getReplicaId();
-    RegionLocations locs = findAllLocationsOrFail(action, true);
-    if (locs == null) return null; // manageError already called
-    HRegionLocation loc = locs.getRegionLocation(replicaId);
-    if (loc == null || loc.getServerName() == null) {
-      locs = findAllLocationsOrFail(action, false);
-      if (locs == null) return null; // manageError already called
-      loc = locs.getRegionLocation(replicaId);
-    }
-    if (loc == null || loc.getServerName() == null) {
-      manageLocationError(action, null);
-      return null;
-    }
-    return loc;
-  }
-
-  private void manageLocationError(Action action, Exception ex) {
-    String msg = "Cannot get replica " + action.getReplicaId()
-        + " location for " + action.getAction();
-    LOG.error(msg);
-    if (ex == null) {
-      ex = new IOException(msg);
-    }
-    manageError(action.getOriginalIndex(), action.getAction(),
-        Retry.NO_LOCATION_PROBLEM, ex, null);
-  }
-
-  private RegionLocations findAllLocationsOrFail(Action action, boolean useCache) {
-    if (action.getAction() == null) throw new IllegalArgumentException("#" + asyncProcess.id +
-        ", row cannot be null");
-    RegionLocations loc = null;
-    try {
-      loc = asyncProcess.connection.locateRegion(
-          tableName, action.getAction().getRow(), useCache, true, action.getReplicaId());
-    } catch (IOException ex) {
-      manageLocationError(action, ex);
-    }
-    return loc;
-  }
-
-  /**
-   * Send a multi action structure to the servers, after a delay depending on the attempt
-   * number. Asynchronous.
-   *
-   * @param actionsByServer the actions structured by regions
-   * @param numAttempt the attempt number.
-   * @param actionsForReplicaThread original actions for replica thread; null on non-first call.
-   */
-  void sendMultiAction(Map<ServerName, MultiAction> actionsByServer,
-                               int numAttempt, List<Action> actionsForReplicaThread, boolean reuseThread) {
-    // Run the last item on the same thread if we are already on a send thread.
-    // We hope most of the time it will be the only item, so we can cut down on threads.
-    int actionsRemaining = actionsByServer.size();
-    // This iteration is by server (the HRegionLocation comparator is by server portion only).
-    for (Map.Entry<ServerName, MultiAction> e : actionsByServer.entrySet()) {
-      ServerName server = e.getKey();
-      MultiAction multiAction = e.getValue();
-      Collection<? extends Runnable> runnables = getNewMultiActionRunnable(server, multiAction,
-          numAttempt);
-      // make sure we correctly count the number of runnables before we try to reuse the send
-      // thread, in case we had to split the request into different runnables because of backoff
-      if (runnables.size() > actionsRemaining) {
-        actionsRemaining = runnables.size();
-      }
-
-      // run all the runnables
-      // HBASE-17475: Do not reuse the thread after stack reach a certain depth to prevent stack overflow
-      // for now, we use HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER to control the depth
-      for (Runnable runnable : runnables) {
-        if ((--actionsRemaining == 0) && reuseThread
-            && numAttempt % HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER != 0) {
-          runnable.run();
-        } else {
-          try {
-            pool.submit(runnable);
-          } catch (Throwable t) {
-            if (t instanceof RejectedExecutionException) {
-              // This should never happen. But as the pool is provided by the end user,
-              // let's secure this a little.
-              LOG.warn("id=" + asyncProcess.id + ", task rejected by pool. Unexpected." +
-                  " Server=" + server.getServerName(), t);
-            } else {
-              // see #HBASE-14359 for more details
-              LOG.warn("Caught unexpected exception/error: ", t);
-            }
-            asyncProcess.decTaskCounters(multiAction.getRegions(), server);
-            // We're likely to fail again, but this will increment the attempt counter,
-            // so it will finish.
-            receiveGlobalFailure(multiAction, server, numAttempt, t);
-          }
-        }
-      }
-    }
-
-    if (actionsForReplicaThread != null) {
-      startWaitingForReplicaCalls(actionsForReplicaThread);
-    }
-  }
-
-  private Collection<? extends Runnable> getNewMultiActionRunnable(ServerName server,
-                                                                   MultiAction multiAction,
-                                                                   int numAttempt) {
-    // no stats to manage, just do the standard action
-    if (asyncProcess.connection.getStatisticsTracker() == null) {
-      if (asyncProcess.connection.getConnectionMetrics() != null) {
-        asyncProcess.connection.getConnectionMetrics().incrNormalRunners();
-      }
-      asyncProcess.incTaskCounters(multiAction.getRegions(), server);
-      SingleServerRequestRunnable runnable = createSingleServerRequest(
-              multiAction, numAttempt, server, callsInProgress);
-      Tracer tracer = Tracer.curThreadTracer();
-
-      if (tracer == null) {
-        return Collections.singletonList(runnable);
-      } else {
-        return Collections.singletonList(tracer.wrap(runnable, "AsyncProcess.sendMultiAction"));
-      }
-    }
-
-    // group the actions by the amount of delay
-    Map<Long, DelayingRunner> actions = new HashMap<>(multiAction.size());
-
-    // split up the actions
-    for (Map.Entry<byte[], List<Action>> e : multiAction.actions.entrySet()) {
-      Long backoff = getBackoff(server, e.getKey());
-      DelayingRunner runner = actions.get(backoff);
-      if (runner == null) {
-        actions.put(backoff, new DelayingRunner(backoff, e));
-      } else {
-        runner.add(e);
-      }
-    }
-
-    List<Runnable> toReturn = new ArrayList<>(actions.size());
-    for (DelayingRunner runner : actions.values()) {
-      asyncProcess.incTaskCounters(runner.getActions().getRegions(), server);
-      String traceText = "AsyncProcess.sendMultiAction";
-      Runnable runnable = createSingleServerRequest(runner.getActions(), numAttempt, server, callsInProgress);
-      // use a delay runner only if we need to sleep for some time
-      if (runner.getSleepTime() > 0) {
-        runner.setRunner(runnable);
-        traceText = "AsyncProcess.clientBackoff.sendMultiAction";
-        runnable = runner;
-        if (asyncProcess.connection.getConnectionMetrics() != null) {
-          asyncProcess.connection.getConnectionMetrics()
-            .incrDelayRunnersAndUpdateDelayInterval(runner.getSleepTime());
-        }
-      } else {
-        if (asyncProcess.connection.getConnectionMetrics() != null) {
-          asyncProcess.connection.getConnectionMetrics().incrNormalRunners();
-        }
-      }
-      runnable = TraceUtil.wrap(runnable, traceText);
-      toReturn.add(runnable);
-
-    }
-    return toReturn;
-  }
-
-  /**
-   * @param server server location where the target region is hosted
-   * @param regionName name of the region which we are going to write some data
-   * @return the amount of time the client should wait until it submit a request to the
-   * specified server and region
-   */
-  private Long getBackoff(ServerName server, byte[] regionName) {
-    ServerStatisticTracker tracker = asyncProcess.connection.getStatisticsTracker();
-    ServerStatistics stats = tracker.getStats(server);
-    return asyncProcess.connection.getBackoffPolicy()
-        .getBackoffTime(server, regionName, stats);
-  }
-
-  /**
-   * Starts waiting to issue replica calls on a different thread; or issues them immediately.
-   */
-  private void startWaitingForReplicaCalls(List<Action> actionsForReplicaThread) {
-    long startTime = EnvironmentEdgeManager.currentTime();
-    ReplicaCallIssuingRunnable replicaRunnable = new ReplicaCallIssuingRunnable(
-        actionsForReplicaThread, startTime);
-    if (asyncProcess.primaryCallTimeoutMicroseconds == 0) {
-      // Start replica calls immediately.
-      replicaRunnable.run();
-    } else {
-      // Start the thread that may kick off replica gets.
-      // TODO: we could do it on the same thread, but it's a user thread, might be a bad idea.
-      try {
-        pool.submit(replicaRunnable);
-      } catch (RejectedExecutionException ree) {
-        LOG.warn("id=" + asyncProcess.id + " replica task rejected by pool; no replica calls", ree);
-      }
-    }
-  }
-
-  /**
-   * Check that we can retry acts accordingly: logs, set the error status.
-   *
-   * @param originalIndex the position in the list sent
-   * @param row           the row
-   * @param canRetry      if false, we won't retry whatever the settings.
-   * @param throwable     the throwable, if any (can be null)
-   * @param server        the location, if any (can be null)
-   * @return true if the action can be retried, false otherwise.
-   */
-  Retry manageError(int originalIndex, Row row, Retry canRetry,
-                                        Throwable throwable, ServerName server) {
-    if (canRetry == Retry.YES
-        && throwable != null && throwable instanceof DoNotRetryIOException) {
-      canRetry = Retry.NO_NOT_RETRIABLE;
-    }
-
-    if (canRetry != Retry.YES) {
-      // Batch.Callback<Res> was not called on failure in 0.94. We keep this.
-      setError(originalIndex, row, throwable, server);
-    } else if (isActionComplete(originalIndex, row)) {
-      canRetry = Retry.NO_OTHER_SUCCEEDED;
-    }
-    return canRetry;
-  }
-
-  /**
-   * Resubmit all the actions from this multiaction after a failure.
-   *
-   * @param rsActions  the actions still to do from the initial list
-   * @param server   the destination
-   * @param numAttempt the number of attempts so far
-   * @param t the throwable (if any) that caused the resubmit
-   */
-  private void receiveGlobalFailure(
-      MultiAction rsActions, ServerName server, int numAttempt, Throwable t) {
-    errorsByServer.reportServerError(server);
-    Retry canRetry = errorsByServer.canTryMore(numAttempt)
-        ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED;
-
-    cleanServerCache(server, t);
-    int failed = 0;
-    int stopped = 0;
-    List<Action> toReplay = new ArrayList<>();
-    for (Map.Entry<byte[], List<Action>> e : rsActions.actions.entrySet()) {
-      byte[] regionName = e.getKey();
-      byte[] row = e.getValue().get(0).getAction().getRow();
-      // Do not use the exception for updating cache because it might be coming from
-      // any of the regions in the MultiAction.
-      updateCachedLocations(server, regionName, row,
-        ClientExceptionsUtil.isMetaClearingException(t) ? null : t);
-      for (Action action : e.getValue()) {
-        Retry retry = manageError(
-            action.getOriginalIndex(), action.getAction(), canRetry, t, server);
-        if (retry == Retry.YES) {
-          toReplay.add(action);
-        } else if (retry == Retry.NO_OTHER_SUCCEEDED) {
-          ++stopped;
-        } else {
-          ++failed;
-        }
-      }
-    }
-
-    if (toReplay.isEmpty()) {
-      logNoResubmit(server, numAttempt, rsActions.size(), t, failed, stopped);
-    } else {
-      resubmit(server, toReplay, numAttempt, rsActions.size(), t);
-    }
-  }
-
-  /**
-   * Log as much info as possible, and, if there is something to replay,
-   * submit it again after a back off sleep.
-   */
-  private void resubmit(ServerName oldServer, List<Action> toReplay,
-                        int numAttempt, int failureCount, Throwable throwable) {
-    // We have something to replay. We're going to sleep a little before.
-
-    // We have two contradicting needs here:
-    //  1) We want to get the new location after having slept, as it may change.
-    //  2) We want to take into account the location when calculating the sleep time.
-    //  3) If all this is just because the response needed to be chunked try again FAST.
-    // It should be possible to have some heuristics to take the right decision. Short term,
-    //  we go for one.
-    boolean retryImmediately = throwable instanceof RetryImmediatelyException;
-    int nextAttemptNumber = retryImmediately ? numAttempt : numAttempt + 1;
-    long backOffTime;
-    if (retryImmediately) {
-      backOffTime = 0;
-    } else if (throwable instanceof CallQueueTooBigException) {
-      // Give a special check on CQTBE, see #HBASE-17114
-      backOffTime = errorsByServer.calculateBackoffTime(oldServer, asyncProcess.pauseForCQTBE);
-    } else {
-      backOffTime = errorsByServer.calculateBackoffTime(oldServer, asyncProcess.pause);
-    }
-    if (numAttempt > asyncProcess.startLogErrorsCnt) {
-      // We use this value to have some logs when we have multiple failures, but not too many
-      //  logs, as errors are to be expected when a region moves, splits and so on
-      LOG.info(createLog(numAttempt, failureCount, toReplay.size(),
-          oldServer, throwable, backOffTime, true, null, -1, -1));
-    }
-
-    try {
-      if (backOffTime > 0) {
-        Thread.sleep(backOffTime);
-      }
-    } catch (InterruptedException e) {
-      LOG.warn("#" + asyncProcess.id + ", not sent: " + toReplay.size() + " operations, " + oldServer, e);
-      Thread.currentThread().interrupt();
-      return;
-    }
-
-    groupAndSendMultiAction(toReplay, nextAttemptNumber);
-  }
-
-  private void logNoResubmit(ServerName oldServer, int numAttempt,
-                             int failureCount, Throwable throwable, int failed, int stopped) {
-    if (failureCount != 0 || numAttempt > asyncProcess.startLogErrorsCnt + 1) {
-      String timeStr = new Date(errorsByServer.getStartTrackingTime()).toString();
-      String logMessage = createLog(numAttempt, failureCount, 0, oldServer,
-          throwable, -1, false, timeStr, failed, stopped);
-      if (failed != 0) {
-        // Only log final failures as warning
-        LOG.warn(logMessage);
-      } else {
-        LOG.info(logMessage);
-      }
-    }
-  }
-
-  /**
-   * Called when we receive the result of a server query.
-   *
-   * @param multiAction    - the multiAction we sent
-   * @param server       - the location. It's used as a server name.
-   * @param responses      - the response, if any
-   * @param numAttempt     - the attempt
-   */
-  private void receiveMultiAction(MultiAction multiAction, ServerName server,
-      MultiResponse responses, int numAttempt) {
-    assert responses != null;
-    updateStats(server, responses);
-    // Success or partial success
-    // Analyze detailed results. We can still have individual failures to be redo.
-    // two specific throwables are managed:
-    //  - DoNotRetryIOException: we continue to retry for other actions
-    //  - RegionMovedException: we update the cache with the new region location
-    Map<byte[], MultiResponse.RegionResult> results = responses.getResults();
-    List<Action> toReplay = new ArrayList<>();
-    Throwable lastException = null;
-    int failureCount = 0;
-    int failed = 0;
-    int stopped = 0;
-    Retry retry = null;
-    // Go by original action.
-    for (Map.Entry<byte[], List<Action>> regionEntry : multiAction.actions.entrySet()) {
-      byte[] regionName = regionEntry.getKey();
-
-      Throwable regionException = responses.getExceptions().get(regionName);
-      if (regionException != null) {
-        cleanServerCache(server, regionException);
-      }
-
-      Map<Integer, Object> regionResults =
-        results.containsKey(regionName) ? results.get(regionName).result : Collections.emptyMap();
-      boolean regionFailureRegistered = false;
-      for (Action sentAction : regionEntry.getValue()) {
-        Object result = regionResults.get(sentAction.getOriginalIndex());
-        if (result == null) {
-          if (regionException == null) {
-            LOG.error("Server sent us neither results nor exceptions for "
-              + Bytes.toStringBinary(regionName)
-              + ", numAttempt:" + numAttempt);
-            regionException = new RuntimeException("Invalid response");
-          }
-          // If the row operation encounters the region-lever error, the exception of action may be
-          // null.
-          result = regionException;
-        }
-        // Failure: retry if it's make sense else update the errors lists
-        if (result instanceof Throwable) {
-          Throwable actionException = (Throwable) result;
-          Row row = sentAction.getAction();
-          lastException = regionException != null ? regionException
-            : ClientExceptionsUtil.findException(actionException);
-          // Register corresponding failures once per server/once per region.
-          if (!regionFailureRegistered) {
-            regionFailureRegistered = true;
-            updateCachedLocations(server, regionName, row.getRow(), actionException);
-          }
-          if (retry == null) {
-            errorsByServer.reportServerError(server);
-            // We determine canRetry only once for all calls, after reporting server failure.
-            retry = errorsByServer.canTryMore(numAttempt) ?
-              Retry.YES : Retry.NO_RETRIES_EXHAUSTED;
-          }
-          ++failureCount;
-          switch (manageError(sentAction.getOriginalIndex(), row, retry, actionException,
-            server)) {
-            case YES:
-              toReplay.add(sentAction);
-              break;
-            case NO_OTHER_SUCCEEDED:
-              ++stopped;
-              break;
-            default:
-              ++failed;
-              break;
-          }
-        } else {
-          invokeCallBack(regionName, sentAction.getAction().getRow(), (CResult) result);
-          setResult(sentAction, result);
-        }
-      }
-    }
-    if (toReplay.isEmpty()) {
-      logNoResubmit(server, numAttempt, failureCount, lastException, failed, stopped);
-    } else {
-      resubmit(server, toReplay, numAttempt, failureCount, lastException);
-    }
-  }
-
-  private void updateCachedLocations(ServerName server, byte[] regionName, byte[] row,
-    Throwable rowException) {
-    if (tableName == null) {
-      return;
-    }
-    try {
-      asyncProcess.connection
-        .updateCachedLocations(tableName, regionName, row, rowException, server);
-    } catch (Throwable ex) {
-      // That should never happen, but if it did, we want to make sure
-      // we still process errors
-      LOG.error("Couldn't update cached region locations: " + ex);
-    }
-  }
-
-  private void invokeCallBack(byte[] regionName, byte[] row, CResult result) {
-    if (callback != null) {
-      try {
-        //noinspection unchecked
-        // TODO: would callback expect a replica region name if it gets one?
-        this.callback.update(regionName, row, result);
-      } catch (Throwable t) {
-        LOG.error("User callback threw an exception for "
-          + Bytes.toStringBinary(regionName) + ", ignoring", t);
-      }
-    }
-  }
-
-  private void cleanServerCache(ServerName server, Throwable regionException) {
-    if (ClientExceptionsUtil.isMetaClearingException(regionException)) {
-      // We want to make sure to clear the cache in case there were location-related exceptions.
-      // We don't to clear the cache for every possible exception that comes through, however.
-      asyncProcess.connection.clearCaches(server);
-    }
-  }
-
-  @VisibleForTesting
-  protected void updateStats(ServerName server, MultiResponse resp) {
-    ConnectionUtils.updateStats(Optional.ofNullable(asyncProcess.connection.getStatisticsTracker()),
-      Optional.ofNullable(asyncProcess.connection.getConnectionMetrics()), server, resp);
-  }
-
-
-  private String createLog(int numAttempt, int failureCount, int replaySize, ServerName sn,
-                           Throwable error, long backOffTime, boolean willRetry, String startTime,
-                           int failed, int stopped) {
-    StringBuilder sb = new StringBuilder();
-    sb.append("id=").append(asyncProcess.id).append(", table=").append(tableName).
-        append(", attempt=").append(numAttempt).append("/").append(asyncProcess.numTries).
-        append(", ");
-
-    if (failureCount > 0 || error != null){
-      sb.append("failureCount=").append(failureCount).append("ops").append(", last exception=").
-          append(error);
-    } else {
-      sb.append("succeeded");
-    }
-
-    sb.append(" on ").append(sn).append(", tracking started ").append(startTime);
-
-    if (willRetry) {
-      sb.append(", retrying after=").append(backOffTime).append("ms").
-          append(", operationsToReplay=").append(replaySize);
-    } else if (failureCount > 0) {
-      if (stopped > 0) {
-        sb.append("; NOT retrying, stopped=").append(stopped).
-            append(" because successful operation on other replica");
-      }
-      if (failed > 0) {
-        sb.append("; NOT retrying, failed=").append(failed).append(" -- final attempt!");
-      }
-    }
-
-    return sb.toString();
-  }
-
-  /**
-   * Sets the non-error result from a particular action.
-   * @param action Action (request) that the server responded to.
-   * @param result The result.
-   */
-  private void setResult(Action action, Object result) {
-    if (result == null) {
-      throw new RuntimeException("Result cannot be null");
-    }
-    ReplicaResultState state = null;
-    boolean isStale = !RegionReplicaUtil.isDefaultReplica(action.getReplicaId());
-    int index = action.getOriginalIndex();
-    if (results == null) {
-      decActionCounter(index);
-      return; // Simple case, no replica requests.
-    }
-    state = trySetResultSimple(index, action.getAction(), false, result, null, isStale);
-    if (state == null) {
-      return; // Simple case, no replica requests.
-    }
-    // At this point we know that state is set to replica tracking class.
-    // It could be that someone else is also looking at it; however, we know there can
-    // only be one state object, and only one thread can set callCount to 0. Other threads
-    // will either see state with callCount 0 after locking it; or will not see state at all
-    // we will replace it with the result.
-    synchronized (state) {
-      if (state.callCount == 0) {
-        return; // someone already set the result
-      }
-      state.callCount = 0;
-    }
-    synchronized (replicaResultLock) {
-      if (results[index] != state) {
-        throw new AssertionError("We set the callCount but someone else replaced the result");
-      }
-      updateResult(index, result);
-    }
-
-    decActionCounter(index);
-  }
-
-  /**
-   * Sets the error from a particular action.
-   * @param index Original action index.
-   * @param row Original request.
-   * @param throwable The resulting error.
-   * @param server The source server.
-   */
-  private void setError(int index, Row row, Throwable throwable, ServerName server) {
-    ReplicaResultState state = null;
-    if (results == null) {
-      // Note that we currently cannot have replica requests with null results. So it shouldn't
-      // happen that multiple replica calls will call dAC for same actions with results == null.
-      // Only one call per action should be present in this case.
-      errors.add(throwable, row, server);
-      decActionCounter(index);
-      return; // Simple case, no replica requests.
-    }
-    state = trySetResultSimple(index, row, true, throwable, server, false);
-    if (state == null) {
-      return; // Simple case, no replica requests.
-    }
-    BatchErrors target = null; // Error will be added to final errors, or temp replica errors.
-    boolean isActionDone = false;
-    synchronized (state) {
-      switch (state.callCount) {
-        case 0: return; // someone already set the result
-        case 1: { // All calls failed, we are the last error.
-          target = errors;
-          isActionDone = true;
-          break;
-        }
-        default: {
-          assert state.callCount > 1;
-          if (state.replicaErrors == null) {
-            state.replicaErrors = new BatchErrors();
-          }
-          target = state.replicaErrors;
-          break;
-        }
-      }
-      --state.callCount;
-    }
-    target.add(throwable, row, server);
-    if (isActionDone) {
-      if (state.replicaErrors != null) { // last call, no need to lock
-        errors.merge(state.replicaErrors);
-      }
-      // See setResult for explanations.
-      synchronized (replicaResultLock) {
-        if (results[index] != state) {
-          throw new AssertionError("We set the callCount but someone else replaced the result");
-        }
-        updateResult(index, throwable);
-      }
-      decActionCounter(index);
-    }
-  }
-
-  /**
-   * Checks if the action is complete; used on error to prevent needless retries.
-   * Does not synchronize, assuming element index/field accesses are atomic.
-   * This is an opportunistic optimization check, doesn't have to be strict.
-   * @param index Original action index.
-   * @param row Original request.
-   */
-  private boolean isActionComplete(int index, Row row) {
-    if (!AsyncProcess.isReplicaGet(row)) return false;
-    Object resObj = results[index];
-    return (resObj != null) && (!(resObj instanceof ReplicaResultState)
-        || ((ReplicaResultState)resObj).callCount == 0);
-  }
-
-  /**
-   * Tries to set the result or error for a particular action as if there were no replica calls.
-   * @return null if successful; replica state if there were in fact replica calls.
-   */
-  private ReplicaResultState trySetResultSimple(int index, Row row, boolean isError,
-                                                Object result, ServerName server, boolean isFromReplica) {
-    Object resObj = null;
-    if (!AsyncProcess.isReplicaGet(row)) {
-      if (isFromReplica) {
-        throw new AssertionError("Unexpected stale result for " + row);
-      }
-      updateResult(index, result);
-    } else {
-      synchronized (replicaResultLock) {
-        resObj = results[index];
-        if (resObj == null) {
-          if (isFromReplica) {
-            throw new AssertionError("Unexpected stale result for " + row);
-          }
-          updateResult(index, result);
-        }
-      }
-    }
-
-    ReplicaResultState rrs =
-        (resObj instanceof ReplicaResultState) ? (ReplicaResultState)resObj : null;
-    if (rrs == null && isError) {
-      // The resObj is not replica state (null or already set).
-      errors.add((Throwable)result, row, server);
-    }
-
-    if (resObj == null) {
-      // resObj is null - no replica calls were made.
-      decActionCounter(index);
-      return null;
-    }
-    return rrs;
-  }
-
-  private void decActionCounter(int index) {
-    long actionsRemaining = actionsInProgress.decrementAndGet();
-    if (actionsRemaining < 0) {
-      String error = buildDetailedErrorMsg("Incorrect actions in progress", index);
-      throw new AssertionError(error);
-    } else if (actionsRemaining == 0) {
-      synchronized (actionsInProgress) {
-        actionsInProgress.notifyAll();
-      }
-    }
-  }
-
-  private String buildDetailedErrorMsg(String string, int index) {
-    StringBuilder error = new StringBuilder(128);
-    error.append(string).append("; called for ").append(index).append(", actionsInProgress ")
-        .append(actionsInProgress.get()).append("; replica gets: ");
-    if (replicaGetIndices != null) {
-      for (int i = 0; i < replicaGetIndices.length; ++i) {
-        error.append(replicaGetIndices[i]).append(", ");
-      }
-    } else {
-      error.append(hasAnyReplicaGets ? "all" : "none");
-    }
-    error.append("; results ");
-    if (results != null) {
-      for (int i = 0; i < results.length; ++i) {
-        Object o = results[i];
-        error.append(((o == null) ? "null" : o.toString())).append(", ");
-      }
-    }
-    return error.toString();
-  }
-
-  @Override
-  public void waitUntilDone() throws InterruptedIOException {
-    try {
-      waitUntilDone(Long.MAX_VALUE);
-    } catch (InterruptedException iex) {
-      throw new InterruptedIOException(iex.getMessage());
-    } finally {
-      if (callsInProgress != null) {
-        for (CancellableRegionServerCallable clb : callsInProgress) {
-          clb.cancel();
-        }
-      }
-    }
-  }
-
-  private boolean waitUntilDone(long cutoff) throws InterruptedException {
-    boolean hasWait = cutoff != Long.MAX_VALUE;
-    long lastLog = EnvironmentEdgeManager.currentTime();
-    long currentInProgress;
-    while (0 != (currentInProgress = actionsInProgress.get())) {
-      long now = EnvironmentEdgeManager.currentTime();
-      if (hasWait && (now * 1000L) > cutoff) {
-        return false;
-      }
-      if (!hasWait) { // Only log if wait is infinite.
-        if (now > lastLog + 10000) {
-          lastLog = now;
-          LOG.info("#" + asyncProcess.id + ", waiting for " + currentInProgress
-              + "  actions to finish on table: " + tableName);
-        }
-      }
-      synchronized (actionsInProgress) {
-        if (actionsInProgress.get() == 0) break;
-        if (!hasWait) {
-          actionsInProgress.wait(10);
-        } else {
-          long waitMicroSecond = Math.min(100000L, (cutoff - now * 1000L));
-          TimeUnit.MICROSECONDS.timedWait(actionsInProgress, waitMicroSecond);
-        }
-      }
-    }
-    return true;
-  }
-
-  @Override
-  public boolean hasError() {
-    return errors.hasErrors();
-  }
-
-  @Override
-  public List<? extends Row> getFailedOperations() {
-    return errors.actions;
-  }
-
-  @Override
-  public RetriesExhaustedWithDetailsException getErrors() {
-    return errors.makeException(asyncProcess.logBatchErrorDetails);
-  }
-
-  @Override
-  public Object[] getResults() throws InterruptedIOException {
-    waitUntilDone();
-    return results;
-  }
-
-  /**
-   * Creates the server error tracker to use inside process.
-   * Currently, to preserve the main assumption about current retries, and to work well with
-   * the retry-limit-based calculation, the calculation is local per Process object.
-   * We may benefit from connection-wide tracking of server errors.
-   * @return ServerErrorTracker to use, null if there is no ServerErrorTracker on this connection
-   */
-  private ConnectionImplementation.ServerErrorTracker createServerErrorTracker() {
-    return new ConnectionImplementation.ServerErrorTracker(
-        asyncProcess.serverTrackerTimeout, asyncProcess.numTries);
-  }
-
-  /**
-   * Create a callable. Isolated to be easily overridden in the tests.
-   */
-  private MultiServerCallable createCallable(final ServerName server, TableName tableName,
-      final MultiAction multi) {
-    return new MultiServerCallable(asyncProcess.connection, tableName, server,
-        multi, asyncProcess.rpcFactory.newController(), rpcTimeout, tracker, multi.getPriority());
-  }
-
-  private void updateResult(int index, Object result) {
-    Object current = results[index];
-    if (current != null) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("The result is assigned repeatedly! current:" + current
-          + ", new:" + result);
-      }
-    }
-    results[index] = result;
-  }
-
-  @VisibleForTesting
-  long getNumberOfActionsInProgress() {
-    return actionsInProgress.get();
-  }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BatchErrors.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BatchErrors.java
deleted file mode 100644
index d3cdc74..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BatchErrors.java
+++ /dev/null
@@ -1,69 +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 org.apache.hadoop.hbase.ServerName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.List;
-
-class BatchErrors {
-  private static final Logger LOG = LoggerFactory.getLogger(BatchErrors.class);
-  final List<Throwable> throwables = new ArrayList<>();
-  final List<Row> actions = new ArrayList<>();
-  final List<String> addresses = new ArrayList<>();
-
-  public synchronized void add(Throwable ex, Row row, ServerName serverName) {
-    if (row == null){
-      throw new IllegalArgumentException("row cannot be null. location=" + serverName);
-    }
-
-    throwables.add(ex);
-    actions.add(row);
-    addresses.add(serverName != null ? serverName.toString() : "null");
-  }
-
-  public boolean hasErrors() {
-    return !throwables.isEmpty();
-  }
-
-  synchronized RetriesExhaustedWithDetailsException makeException(boolean logDetails) {
-    if (logDetails) {
-      LOG.error("Exception occurred! Exception details: " + throwables + ";\nActions: "
-              + actions);
-    }
-    return new RetriesExhaustedWithDetailsException(new ArrayList<>(throwables),
-            new ArrayList<>(actions), new ArrayList<>(addresses));
-  }
-
-  public synchronized void clear() {
-    throwables.clear();
-    actions.clear();
-    addresses.clear();
-  }
-
-  public synchronized void merge(BatchErrors other) {
-    throwables.addAll(other.throwables);
-    actions.addAll(other.actions);
-    addresses.addAll(other.addresses);
-  }
-}
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
deleted file mode 100644
index 6d70219..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
+++ /dev/null
@@ -1,500 +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 static org.apache.hadoop.hbase.client.BufferedMutatorParams.UNSET;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
-/**
- * <p>
- * Used to communicate with a single HBase table similar to {@link Table}
- * but meant for batched, potentially asynchronous puts. Obtain an instance from
- * a {@link Connection} and call {@link #close()} afterwards. Provide an alternate
- * to this implementation by setting {@link BufferedMutatorParams#implementationClassName(String)}
- * or by setting alternate classname via the key {} in Configuration.
- * </p>
- *
- * <p>
- * While this can be used across threads, great care should be used when doing so.
- * Errors are global to the buffered mutator and the Exceptions can be thrown on any
- * thread that causes the flush for requests.
- * </p>
- *
- * @see ConnectionFactory
- * @see Connection
- * @since 1.0.0
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-class BufferedMutatorImpl implements BufferedMutator {
-
-  private static final Logger LOG = LoggerFactory.getLogger(BufferedMutatorImpl.class);
-
-  private final ExceptionListener listener;
-
-  private final TableName tableName;
-
-  private final Configuration conf;
-  private final ConcurrentLinkedQueue<Mutation> writeAsyncBuffer = new ConcurrentLinkedQueue<>();
-  private final AtomicLong currentWriteBufferSize = new AtomicLong(0);
-  /**
-   * Count the size of {@link BufferedMutatorImpl#writeAsyncBuffer}.
-   * The {@link ConcurrentLinkedQueue#size()} is NOT a constant-time operation.
-   */
-  private final AtomicInteger undealtMutationCount = new AtomicInteger(0);
-  private final long writeBufferSize;
-
-  private final AtomicLong writeBufferPeriodicFlushTimeoutMs = new AtomicLong(0);
-  private final AtomicLong writeBufferPeriodicFlushTimerTickMs =
-          new AtomicLong(MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS);
-  private Timer writeBufferPeriodicFlushTimer = null;
-
-  private final int maxKeyValueSize;
-  private final ExecutorService pool;
-  private final AtomicInteger rpcTimeout;
-  private final AtomicInteger operationTimeout;
-  private final boolean cleanupPoolOnClose;
-  private volatile boolean closed = false;
-  private final AsyncProcess ap;
-
-  @VisibleForTesting
-  BufferedMutatorImpl(ConnectionImplementation conn, BufferedMutatorParams params,
-      AsyncProcess ap) {
-    if (conn == null || conn.isClosed()) {
-      throw new IllegalArgumentException("Connection is null or closed.");
-    }
-    this.tableName = params.getTableName();
-    this.conf = conn.getConfiguration();
-    this.listener = params.getListener();
-    if (params.getPool() == null) {
-      this.pool = HTable.getDefaultExecutor(conf);
-      cleanupPoolOnClose = true;
-    } else {
-      this.pool = params.getPool();
-      cleanupPoolOnClose = false;
-    }
-    ConnectionConfiguration tableConf = new ConnectionConfiguration(conf);
-    this.writeBufferSize =
-            params.getWriteBufferSize() != UNSET ?
-            params.getWriteBufferSize() : tableConf.getWriteBufferSize();
-
-    // Set via the setter because it does value validation and starts/stops the TimerTask
-    long newWriteBufferPeriodicFlushTimeoutMs =
-            params.getWriteBufferPeriodicFlushTimeoutMs() != UNSET
-              ? params.getWriteBufferPeriodicFlushTimeoutMs()
-              : tableConf.getWriteBufferPeriodicFlushTimeoutMs();
-    long newWriteBufferPeriodicFlushTimerTickMs =
-            params.getWriteBufferPeriodicFlushTimerTickMs() != UNSET
-              ? params.getWriteBufferPeriodicFlushTimerTickMs()
-              : tableConf.getWriteBufferPeriodicFlushTimerTickMs();
-    this.setWriteBufferPeriodicFlush(
-            newWriteBufferPeriodicFlushTimeoutMs,
-            newWriteBufferPeriodicFlushTimerTickMs);
-
-    this.maxKeyValueSize =
-            params.getMaxKeyValueSize() != UNSET ?
-            params.getMaxKeyValueSize() : tableConf.getMaxKeyValueSize();
-
-    this.rpcTimeout = new AtomicInteger(
-            params.getRpcTimeout() != UNSET ?
-            params.getRpcTimeout() : conn.getConnectionConfiguration().getWriteRpcTimeout());
-
-    this.operationTimeout = new AtomicInteger(
-            params.getOperationTimeout() != UNSET ?
-            params.getOperationTimeout() : conn.getConnectionConfiguration().getOperationTimeout());
-    this.ap = ap;
-  }
-
-  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.
-      new AsyncProcess(conn, conn.getConfiguration(), rpcCallerFactory, rpcFactory));
-  }
-
-  private void checkClose() {
-    if (closed) {
-      throw new IllegalStateException("Cannot put when the BufferedMutator is closed.");
-    }
-  }
-
-  @VisibleForTesting
-  ExecutorService getPool() {
-    return pool;
-  }
-
-  @VisibleForTesting
-  AsyncProcess getAsyncProcess() {
-    return ap;
-  }
-
-  @Override
-  public TableName getName() {
-    return tableName;
-  }
-
-  @Override
-  public Configuration getConfiguration() {
-    return conf;
-  }
-
-  @Override
-  public void mutate(Mutation m) throws InterruptedIOException,
-      RetriesExhaustedWithDetailsException {
-    mutate(Collections.singletonList(m));
-  }
-
-  @Override
-  public void mutate(List<? extends Mutation> ms) throws InterruptedIOException,
-      RetriesExhaustedWithDetailsException {
-    checkClose();
-
-    long toAddSize = 0;
-    int toAddCount = 0;
-    for (Mutation m : ms) {
-      if (m instanceof Put) {
-        ConnectionUtils.validatePut((Put) m, maxKeyValueSize);
-      }
-      toAddSize += m.heapSize();
-      ++toAddCount;
-    }
-
-    if (currentWriteBufferSize.get() == 0) {
-      firstRecordInBufferTimestamp.set(System.currentTimeMillis());
-    }
-    currentWriteBufferSize.addAndGet(toAddSize);
-    writeAsyncBuffer.addAll(ms);
-    undealtMutationCount.addAndGet(toAddCount);
-    doFlush(false);
-  }
-
-  @VisibleForTesting
-  protected long getExecutedWriteBufferPeriodicFlushes() {
-    return executedWriteBufferPeriodicFlushes.get();
-  }
-
-  private final AtomicLong firstRecordInBufferTimestamp = new AtomicLong(0);
-  private final AtomicLong executedWriteBufferPeriodicFlushes = new AtomicLong(0);
-
-  private void timerCallbackForWriteBufferPeriodicFlush() {
-    if (currentWriteBufferSize.get() == 0) {
-      return; // Nothing to flush
-    }
-    long now = System.currentTimeMillis();
-    if (firstRecordInBufferTimestamp.get() + writeBufferPeriodicFlushTimeoutMs.get() > now) {
-      return; // No need to flush yet
-    }
-    // The first record in the writebuffer has been in there too long --> flush
-    try {
-      executedWriteBufferPeriodicFlushes.incrementAndGet();
-      flush();
-    } catch (InterruptedIOException | RetriesExhaustedWithDetailsException e) {
-      LOG.error("Exception during timerCallbackForWriteBufferPeriodicFlush --> " + e.getMessage());
-    }
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-    if (closed) {
-      return;
-    }
-    // Stop any running Periodic Flush timer.
-    disableWriteBufferPeriodicFlush();
-    try {
-      // As we can have an operation in progress even if the buffer is empty, we call
-      // doFlush at least one time.
-      doFlush(true);
-    } finally {
-      if (cleanupPoolOnClose) {
-        this.pool.shutdown();
-        try {
-          if (!pool.awaitTermination(600, TimeUnit.SECONDS)) {
-            LOG.warn("close() failed to terminate pool after 10 minutes. Abandoning pool.");
-          }
-        } catch (InterruptedException e) {
-          LOG.warn("waitForTermination interrupted");
-          Thread.currentThread().interrupt();
-        }
-      }
-      closed = true;
-    }
-  }
-
-  private AsyncProcessTask createTask(QueueRowAccess access) {
-    return new AsyncProcessTask(AsyncProcessTask.newBuilder()
-        .setPool(pool)
-        .setTableName(tableName)
-        .setRowAccess(access)
-        .setSubmittedRows(AsyncProcessTask.SubmittedRows.AT_LEAST_ONE)
-        .build()) {
-      @Override
-      public int getRpcTimeout() {
-        return rpcTimeout.get();
-      }
-
-      @Override
-      public int getOperationTimeout() {
-        return operationTimeout.get();
-      }
-    };
-  }
-
-  @Override
-  public void flush() throws InterruptedIOException, RetriesExhaustedWithDetailsException {
-    checkClose();
-    doFlush(true);
-  }
-
-  /**
-   * Send the operations in the buffer to the servers.
-   *
-   * @param flushAll - if true, sends all the writes and wait for all of them to finish before
-   *                 returning. Otherwise, flush until buffer size is smaller than threshold
-   */
-  private void doFlush(boolean flushAll) throws InterruptedIOException,
-      RetriesExhaustedWithDetailsException {
-    List<RetriesExhaustedWithDetailsException> errors = new ArrayList<>();
-    while (true) {
-      if (!flushAll && currentWriteBufferSize.get() <= writeBufferSize) {
-        // There is the room to accept more mutations.
-        break;
-      }
-      AsyncRequestFuture asf;
-      try (QueueRowAccess access = createQueueRowAccess()) {
-        if (access.isEmpty()) {
-          // It means someone has gotten the ticker to run the flush.
-          break;
-        }
-        asf = ap.submit(createTask(access));
-      }
-      // DON'T do the wait in the try-with-resources. Otherwise, the undealt mutations won't
-      // be released.
-      asf.waitUntilDone();
-      if (asf.hasError()) {
-        errors.add(asf.getErrors());
-      }
-    }
-
-    RetriesExhaustedWithDetailsException exception = makeException(errors);
-    if (exception == null) {
-      return;
-    } else if(listener == null) {
-      throw exception;
-    } else {
-      listener.onException(exception, this);
-    }
-  }
-
-  private static RetriesExhaustedWithDetailsException makeException(
-    List<RetriesExhaustedWithDetailsException> errors) {
-    switch (errors.size()) {
-      case 0:
-        return null;
-      case 1:
-        return errors.get(0);
-      default:
-        List<Throwable> exceptions = new ArrayList<>();
-        List<Row> actions = new ArrayList<>();
-        List<String> hostnameAndPort = new ArrayList<>();
-        errors.forEach(e -> {
-          exceptions.addAll(e.exceptions);
-          actions.addAll(e.actions);
-          hostnameAndPort.addAll(e.hostnameAndPort);
-        });
-        return new RetriesExhaustedWithDetailsException(exceptions, actions, hostnameAndPort);
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public long getWriteBufferSize() {
-    return this.writeBufferSize;
-  }
-
-  @Override
-  public synchronized void setWriteBufferPeriodicFlush(long timeoutMs, long timerTickMs) {
-    long originalTimeoutMs   = this.writeBufferPeriodicFlushTimeoutMs.get();
-    long originalTimerTickMs = this.writeBufferPeriodicFlushTimerTickMs.get();
-
-    // Both parameters have minimal values.
-    writeBufferPeriodicFlushTimeoutMs.set(Math.max(0, timeoutMs));
-    writeBufferPeriodicFlushTimerTickMs.set(
-            Math.max(MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS, timerTickMs));
-
-    // If something changed we stop the old Timer.
-    if (writeBufferPeriodicFlushTimeoutMs.get() != originalTimeoutMs ||
-        writeBufferPeriodicFlushTimerTickMs.get() != originalTimerTickMs) {
-      if (writeBufferPeriodicFlushTimer != null) {
-        writeBufferPeriodicFlushTimer.cancel();
-        writeBufferPeriodicFlushTimer = null;
-      }
-    }
-
-    // If we have the need for a timer and there is none we start it
-    if (writeBufferPeriodicFlushTimer == null &&
-        writeBufferPeriodicFlushTimeoutMs.get() > 0) {
-      writeBufferPeriodicFlushTimer = new Timer(true); // Create Timer running as Daemon.
-      writeBufferPeriodicFlushTimer.schedule(new TimerTask() {
-        @Override
-        public void run() {
-          BufferedMutatorImpl.this.timerCallbackForWriteBufferPeriodicFlush();
-        }
-      }, writeBufferPeriodicFlushTimerTickMs.get(),
-         writeBufferPeriodicFlushTimerTickMs.get());
-    }
-  }
-
-  @Override
-  public long getWriteBufferPeriodicFlushTimeoutMs() {
-    return writeBufferPeriodicFlushTimeoutMs.get();
-  }
-
-  @Override
-  public long getWriteBufferPeriodicFlushTimerTickMs() {
-    return writeBufferPeriodicFlushTimerTickMs.get();
-  }
-
-  @Override
-  public void setRpcTimeout(int rpcTimeout) {
-    this.rpcTimeout.set(rpcTimeout);
-  }
-
-  @Override
-  public void setOperationTimeout(int operationTimeout) {
-    this.operationTimeout.set(operationTimeout);
-  }
-
-  @VisibleForTesting
-  long getCurrentWriteBufferSize() {
-    return currentWriteBufferSize.get();
-  }
-
-  /**
-   * Count the mutations which haven't been processed.
-   * @return count of undealt mutation
-   */
-  @VisibleForTesting
-  int size() {
-    return undealtMutationCount.get();
-  }
-
-  /**
-   * Count the mutations which haven't been flushed
-   * @return count of unflushed mutation
-   */
-  @VisibleForTesting
-  int getUnflushedSize() {
-    return writeAsyncBuffer.size();
-  }
-
-  @VisibleForTesting
-  QueueRowAccess createQueueRowAccess() {
-    return new QueueRowAccess();
-  }
-
-  @VisibleForTesting
-  class QueueRowAccess implements RowAccess<Row>, Closeable {
-    private int remainder = undealtMutationCount.getAndSet(0);
-    private Mutation last = null;
-
-    private void restoreLastMutation() {
-      // restore the last mutation since it isn't submitted
-      if (last != null) {
-        writeAsyncBuffer.add(last);
-        currentWriteBufferSize.addAndGet(last.heapSize());
-        last = null;
-      }
-    }
-
-    @Override
-    public void close() {
-      restoreLastMutation();
-      if (remainder > 0) {
-        undealtMutationCount.addAndGet(remainder);
-        remainder = 0;
-      }
-    }
-
-    @Override
-    public Iterator<Row> iterator() {
-      return new Iterator<Row>() {
-        private int countDown = remainder;
-        @Override
-        public boolean hasNext() {
-          return countDown > 0;
-        }
-        @Override
-        public Row next() {
-          restoreLastMutation();
-          if (!hasNext()) {
-            throw new NoSuchElementException();
-          }
-          last = writeAsyncBuffer.poll();
-          if (last == null) {
-            throw new NoSuchElementException();
-          }
-          currentWriteBufferSize.addAndGet(-last.heapSize());
-          --countDown;
-          return last;
-        }
-        @Override
-        public void remove() {
-          if (last == null) {
-            throw new IllegalStateException();
-          }
-          --remainder;
-          last = null;
-        }
-      };
-    }
-
-    @Override
-    public int size() {
-      return remainder;
-    }
-
-    @Override
-    public boolean isEmpty() {
-      return remainder <= 0;
-    }
-  }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Cancellable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Cancellable.java
deleted file mode 100644
index 5095c96..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Cancellable.java
+++ /dev/null
@@ -1,31 +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 org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * This should be implemented by the Get/Scan implementations that
- * talk to replica regions. When an RPC response is received from one
- * of the replicas, the RPCs to the other replicas are cancelled.
- */
-@InterfaceAudience.Private
-interface Cancellable {
-  public void cancel();
-  public boolean isCancelled();
-}
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
deleted file mode 100644
index f81018e..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java
+++ /dev/null
@@ -1,123 +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.io.InterruptedIOException;
-
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-
-/**
- * This class is used to unify HTable calls with AsyncProcess Framework. HTable can use
- * AsyncProcess directly though this class. Also adds global timeout tracking on top of
- * RegionServerCallable and implements Cancellable.
- * Global timeout tracking conflicts with logic in RpcRetryingCallerImpl's callWithRetries. So you
- * can only use this callable in AsyncProcess which only uses callWithoutRetries and retries in its
- * own implementation.
- */
-@InterfaceAudience.Private
-abstract class CancellableRegionServerCallable<T> extends ClientServiceCallable<T> implements
-    Cancellable {
-  private final RetryingTimeTracker tracker;
-  private final int rpcTimeout;
-
-  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;
-  }
-
-  /* Override so can mess with the callTimeout.
-   * (non-Javadoc)
-   * @see org.apache.hadoop.hbase.client.RegionServerCallable#rpcCall(int)
-   */
-  @Override
-  public T call(int operationTimeout) throws IOException {
-    if (isCancelled()) return null;
-    if (Thread.interrupted()) {
-      throw new InterruptedIOException();
-    }
-    // It is expected (it seems) that tracker.start can be called multiple times (on each trip
-    // through the call when retrying). Also, we can call start and no need of a stop.
-    this.tracker.start();
-    int remainingTime = tracker.getRemainingTime(operationTimeout);
-    if (remainingTime <= 1) {
-      // "1" is a special return value in RetryingTimeTracker, see its implementation.
-      throw new DoNotRetryIOException("Operation rpcTimeout");
-    }
-    return super.call(Math.min(rpcTimeout, remainingTime));
-  }
-
-  @Override
-  public void prepare(boolean reload) throws IOException {
-    if (isCancelled()) return;
-    if (Thread.interrupted()) {
-      throw new InterruptedIOException();
-    }
-    super.prepare(reload);
-  }
-
-  @Override
-  protected void setStubByServiceName(ServerName serviceName) throws IOException {
-    setStub(getConnection().getClient(serviceName));
-  }
-
-  @Override
-  public void cancel() {
-    getRpcController().startCancel();
-  }
-
-  @Override
-  public boolean isCancelled() {
-    return getRpcController().isCanceled();
-  }
-
-  protected ClientProtos.MultiResponse doMulti(ClientProtos.MultiRequest request)
-  throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
-    return getStub().multi(getRpcController(), request);
-  }
-
-  protected ClientProtos.ScanResponse doScan(ClientProtos.ScanRequest request)
-  throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
-    return getStub().scan(getRpcController(), request);
-  }
-
-  protected ClientProtos.PrepareBulkLoadResponse doPrepareBulkLoad(
-      ClientProtos.PrepareBulkLoadRequest request)
-  throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
-    return getStub().prepareBulkLoad(getRpcController(), request);
-  }
-
-  protected ClientProtos.BulkLoadHFileResponse doBulkLoadHFile(
-      ClientProtos.BulkLoadHFileRequest request)
-  throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
-    return getStub().bulkLoadHFile(getRpcController(), request);
-  }
-
-  protected ClientProtos.CleanupBulkLoadResponse doCleanupBulkLoad(
-      ClientProtos.CleanupBulkLoadRequest request)
-  throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
-    return getStub().cleanupBulkLoad(getRpcController(), request);
-  }
-}
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
deleted file mode 100644
index 76d7409..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
+++ /dev/null
@@ -1,193 +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 org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
-import static org.apache.hadoop.hbase.client.ConnectionUtils.calcEstimatedSize;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.function.Consumer;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.util.Threads;
-
-/**
- * ClientAsyncPrefetchScanner implements async scanner behaviour.
- * Specifically, the cache used by this scanner is a concurrent queue which allows both
- * the producer (hbase client) and consumer (application) to access the queue in parallel.
- * The number of rows returned in a prefetch is defined by the caching factor and the result size
- * factor.
- * This class allocates a buffer cache, whose size is a function of both factors.
- * The prefetch is invoked when the cache is half­filled, instead of waiting for it to be empty.
- * This is defined in the method {@link ClientAsyncPrefetchScanner#prefetchCondition()}.
- */
-@InterfaceAudience.Private
-public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
-
-  private long maxCacheSize;
-  private AtomicLong cacheSizeInBytes;
-  // exception queue (from prefetch to main scan execution)
-  private Queue<Exception> exceptionsQueue;
-  // prefetch thread to be executed asynchronously
-  private Thread prefetcher;
-  // used for testing
-  private Consumer<Boolean> prefetchListener;
-
-  private final Lock lock = new ReentrantLock();
-  private final Condition notEmpty = lock.newCondition();
-  private final Condition notFull = lock.newCondition();
-
-  public ClientAsyncPrefetchScanner(Configuration configuration, Scan scan, TableName name,
-      ConnectionImplementation connection, RpcRetryingCallerFactory rpcCallerFactory,
-      RpcControllerFactory rpcControllerFactory, ExecutorService pool,
-      int replicaCallTimeoutMicroSecondScan) throws IOException {
-    super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool,
-      replicaCallTimeoutMicroSecondScan);
-  }
-
-  @VisibleForTesting
-  void setPrefetchListener(Consumer<Boolean> prefetchListener) {
-    this.prefetchListener = prefetchListener;
-  }
-
-  @Override
-  protected void initCache() {
-    // concurrent cache
-    maxCacheSize = resultSize2CacheSize(maxScannerResultSize);
-    cache = new LinkedBlockingQueue<>();
-    cacheSizeInBytes = new AtomicLong(0);
-    exceptionsQueue = new ConcurrentLinkedQueue<>();
-    prefetcher = new Thread(new PrefetchRunnable());
-    Threads.setDaemonThreadRunning(prefetcher, tableName + ".asyncPrefetcher");
-  }
-
-  private long resultSize2CacheSize(long maxResultSize) {
-    // * 2 if possible
-    return maxResultSize > Long.MAX_VALUE / 2 ? maxResultSize : maxResultSize * 2;
-  }
-
-  @Override
-  public Result next() throws IOException {
-    try {
-      lock.lock();
-      while (cache.isEmpty()) {
-        handleException();
-        if (this.closed) {
-          return null;
-        }
-        try {
-          notEmpty.await();
-        } catch (InterruptedException e) {
-          throw new InterruptedIOException("Interrupted when wait to load cache");
-        }
-      }
-
-      Result result = pollCache();
-      if (prefetchCondition()) {
-        notFull.signalAll();
-      }
-      return result;
-    } finally {
-      lock.unlock();
-      handleException();
-    }
-  }
-
-  @Override
-  public void close() {
-    try {
-      lock.lock();
-      super.close();
-      closed = true;
-      notFull.signalAll();
-      notEmpty.signalAll();
-    } finally {
-      lock.unlock();
-    }
-  }
-
-  @Override
-  protected void addEstimatedSize(long estimatedSize) {
-    cacheSizeInBytes.addAndGet(estimatedSize);
-  }
-
-  private void handleException() throws IOException {
-    //The prefetch task running in the background puts any exception it
-    //catches into this exception queue.
-    // Rethrow the exception so the application can handle it.
-    while (!exceptionsQueue.isEmpty()) {
-      Exception first = exceptionsQueue.peek();
-      if (first instanceof IOException) {
-        throw (IOException) first;
-      }
-      throw (RuntimeException) first;
-    }
-  }
-
-  private boolean prefetchCondition() {
-    return cacheSizeInBytes.get() < maxCacheSize / 2;
-  }
-
-  private Result pollCache() {
-    Result res = cache.poll();
-    long estimatedSize = calcEstimatedSize(res);
-    addEstimatedSize(-estimatedSize);
-    return res;
-  }
-
-  private class PrefetchRunnable implements Runnable {
-
-    @Override
-    public void run() {
-      while (!closed) {
-        boolean succeed = false;
-        try {
-          lock.lock();
-          while (!prefetchCondition()) {
-            notFull.await();
-          }
-          loadCache();
-          succeed = true;
-        } catch (Exception e) {
-          exceptionsQueue.add(e);
-        } finally {
-          notEmpty.signalAll();
-          lock.unlock();
-          if (prefetchListener != null) {
-            prefetchListener.accept(succeed);
-          }
-        }
-      }
-    }
-
-  }
-
-}
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
deleted file mode 100644
index fb89925..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
+++ /dev/null
@@ -1,566 +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 static org.apache.hadoop.hbase.client.ConnectionUtils.calcEstimatedSize;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.createScanResultCache;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.incRegionCountMetrics;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayDeque;
-import java.util.Queue;
-import java.util.concurrent.ExecutorService;
-import org.apache.commons.lang3.mutable.MutableBoolean;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.UnknownScannerException;
-import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults;
-import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
-import org.apache.hadoop.hbase.exceptions.ScannerResetException;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.regionserver.LeaseException;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
-/**
- * Implements the scanner interface for the HBase client. If there are multiple regions in a table,
- * this scanner will iterate through them all.
- */
-@InterfaceAudience.Private
-public abstract class ClientScanner extends AbstractClientScanner {
-
-  private static final Logger LOG = LoggerFactory.getLogger(ClientScanner.class);
-
-  protected final Scan scan;
-  protected boolean closed = false;
-  // Current region scanner is against. Gets cleared if current region goes
-  // wonky: e.g. if it splits on us.
-  protected HRegionInfo currentRegion = null;
-  protected ScannerCallableWithReplicas callable = null;
-  protected Queue<Result> cache;
-  private final ScanResultCache scanResultCache;
-  protected final int caching;
-  protected long lastNext;
-  // Keep lastResult returned successfully in case we have to reset scanner.
-  protected Result lastResult = null;
-  protected final long maxScannerResultSize;
-  private final ConnectionImplementation connection;
-  protected final TableName tableName;
-  protected final int scannerTimeout;
-  protected RpcRetryingCaller<Result[]> caller;
-  protected RpcControllerFactory rpcControllerFactory;
-  protected Configuration conf;
-  // The timeout on the primary. Applicable if there are multiple replicas for a region
-  // In that case, we will only wait for this much timeout on the primary before going
-  // to the replicas and trying the same scan. Note that the retries will still happen
-  // on each replica and the first successful results will be taken. A timeout of 0 is
-  // disallowed.
-  protected final int primaryOperationTimeout;
-  private int retries;
-  protected final ExecutorService pool;
-
-  /**
-   * Create a new ClientScanner for the specified table Note that the passed {@link Scan}'s start
-   * row maybe changed changed.
-   * @param conf The {@link Configuration} to use.
-   * @param scan {@link Scan} to use in this scanner
-   * @param tableName The table that we wish to scan
-   * @param connection Connection identifying the cluster
-   * @throws IOException
-   */
-  public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName,
-      ConnectionImplementation connection, RpcRetryingCallerFactory rpcFactory,
-      RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout)
-      throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(
-        "Scan table=" + tableName + ", startRow=" + Bytes.toStringBinary(scan.getStartRow()));
-    }
-    this.scan = scan;
-    this.tableName = tableName;
-    this.lastNext = System.currentTimeMillis();
-    this.connection = connection;
-    this.pool = pool;
-    this.primaryOperationTimeout = primaryOperationTimeout;
-    this.retries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
-      HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
-    if (scan.getMaxResultSize() > 0) {
-      this.maxScannerResultSize = scan.getMaxResultSize();
-    } else {
-      this.maxScannerResultSize = conf.getLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
-        HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
-    }
-    this.scannerTimeout = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
-        HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
-
-    // check if application wants to collect scan metrics
-    initScanMetrics(scan);
-
-    // Use the caching from the Scan. If not set, use the default cache setting for this table.
-    if (this.scan.getCaching() > 0) {
-      this.caching = this.scan.getCaching();
-    } else {
-      this.caching = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_CACHING,
-        HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
-    }
-
-    this.caller = rpcFactory.<Result[]> newCaller();
-    this.rpcControllerFactory = controllerFactory;
-
-    this.conf = conf;
-
-    this.scanResultCache = createScanResultCache(scan);
-    initCache();
-  }
-
-  protected ConnectionImplementation getConnection() {
-    return this.connection;
-  }
-
-  protected TableName getTable() {
-    return this.tableName;
-  }
-
-  protected int getRetries() {
-    return this.retries;
-  }
-
-  protected int getScannerTimeout() {
-    return this.scannerTimeout;
-  }
-
-  protected Configuration getConf() {
-    return this.conf;
-  }
-
-  protected Scan getScan() {
-    return scan;
-  }
-
-  protected ExecutorService getPool() {
-    return pool;
-  }
-
-  protected int getPrimaryOperationTimeout() {
-    return primaryOperationTimeout;
-  }
-
-  protected int getCaching() {
-    return caching;
-  }
-
-  protected long getTimestamp() {
-    return lastNext;
-  }
-
-  @VisibleForTesting
-  protected long getMaxResultSize() {
-    return maxScannerResultSize;
-  }
-
-  private void closeScanner() throws IOException {
-    if (this.callable != null) {
-      this.callable.setClose();
-      call(callable, caller, scannerTimeout, false);
-      this.callable = null;
-    }
-  }
-
-  /**
-   * Will be called in moveToNextRegion when currentRegion is null. Abstract because for normal
-   * scan, we will start next scan from the endKey of the currentRegion, and for reversed scan, we
-   * will start next scan from the startKey of the currentRegion.
-   * @return {@code false} if we have reached the stop row. Otherwise {@code true}.
-   */
-  protected abstract boolean setNewStartKey();
-
-  /**
-   * Will be called in moveToNextRegion to create ScannerCallable. Abstract because for reversed
-   * scan we need to create a ReversedScannerCallable.
-   */
-  protected abstract ScannerCallable createScannerCallable();
-
-  /**
-   * Close the previous scanner and create a new ScannerCallable for the next scanner.
-   * <p>
-   * Marked as protected only because TestClientScanner need to override this method.
-   * @return false if we should terminate the scan. Otherwise
-   */
-  @VisibleForTesting
-  protected boolean moveToNextRegion() {
-    // Close the previous scanner if it's open
-    try {
-      closeScanner();
-    } catch (IOException e) {
-      // not a big deal continue
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("close scanner for " + currentRegion + " failed", e);
-      }
-    }
-    if (currentRegion != null) {
-      if (!setNewStartKey()) {
-        return false;
-      }
-      scan.resetMvccReadPoint();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Finished " + this.currentRegion);
-      }
-    }
-    if (LOG.isDebugEnabled() && this.currentRegion != null) {
-      // Only worth logging if NOT first region in scan.
-      LOG.debug(
-        "Advancing internal scanner to startKey at '" + Bytes.toStringBinary(scan.getStartRow()) +
-            "', " + (scan.includeStartRow() ? "inclusive" : "exclusive"));
-    }
-    // clear the current region, we will set a new value to it after the first call of the new
-    // callable.
-    this.currentRegion = null;
-    this.callable =
-        new ScannerCallableWithReplicas(getTable(), getConnection(), createScannerCallable(), pool,
-            primaryOperationTimeout, scan, getRetries(), scannerTimeout, caching, conf, caller);
-    this.callable.setCaching(this.caching);
-    incRegionCountMetrics(scanMetrics);
-    return true;
-  }
-
-  @VisibleForTesting
-  boolean isAnyRPCcancelled() {
-    return callable.isAnyRPCcancelled();
-  }
-
-  private Result[] call(ScannerCallableWithReplicas callable, RpcRetryingCaller<Result[]> caller,
-      int scannerTimeout, boolean updateCurrentRegion) throws IOException {
-    if (Thread.interrupted()) {
-      throw new InterruptedIOException();
-    }
-    // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas,
-    // we do a callWithRetries
-    Result[] rrs = caller.callWithoutRetries(callable, scannerTimeout);
-    if (currentRegion == null && updateCurrentRegion) {
-      currentRegion = callable.getHRegionInfo();
-    }
-    return rrs;
-  }
-
-  protected void initSyncCache() {
-    cache = new ArrayDeque<>();
-  }
-
-  protected Result nextWithSyncCache() throws IOException {
-    Result result = cache.poll();
-    if (result != null) {
-      return result;
-    }
-    // If there is nothing left in the cache and the scanner is closed,
-    // return a no-op
-    if (this.closed) {
-      return null;
-    }
-
-    loadCache();
-
-    // try again to load from cache
-    result = cache.poll();
-    return result;
-  }
-
-  @VisibleForTesting
-  public int getCacheSize() {
-    return cache != null ? cache.size() : 0;
-  }
-
-  private boolean scanExhausted(Result[] values) {
-    return callable.moreResultsForScan() == MoreResults.NO;
-  }
-
-  private boolean regionExhausted(Result[] values) {
-    // 1. Not a heartbeat message and we get nothing, this means the region is exhausted. And in the
-    // old time we always return empty result for a open scanner operation so we add a check here to
-    // keep compatible with the old logic. Should remove the isOpenScanner in the future.
-    // 2. Server tells us that it has no more results for this region.
-    return (values.length == 0 && !callable.isHeartbeatMessage()) ||
-        callable.moreResultsInRegion() == MoreResults.NO;
-  }
-
-  private void closeScannerIfExhausted(boolean exhausted) throws IOException {
-    if (exhausted) {
-      closeScanner();
-    }
-  }
-
-  private void handleScanError(DoNotRetryIOException e,
-      MutableBoolean retryAfterOutOfOrderException, int retriesLeft) throws DoNotRetryIOException {
-    // An exception was thrown which makes any partial results that we were collecting
-    // invalid. The scanner will need to be reset to the beginning of a row.
-    scanResultCache.clear();
-
-    // Unfortunately, DNRIOE is used in two different semantics.
-    // (1) The first is to close the client scanner and bubble up the exception all the way
-    // to the application. This is preferred when the exception is really un-recoverable
-    // (like CorruptHFileException, etc). Plain DoNotRetryIOException also falls into this
-    // bucket usually.
-    // (2) Second semantics is to close the current region scanner only, but continue the
-    // client scanner by overriding the exception. This is usually UnknownScannerException,
-    // OutOfOrderScannerNextException, etc where the region scanner has to be closed, but the
-    // application-level ClientScanner has to continue without bubbling up the exception to
-    // the client. See RSRpcServices to see how it throws DNRIOE's.
-    // See also: HBASE-16604, HBASE-17187
-
-    // If exception is any but the list below throw it back to the client; else setup
-    // the scanner and retry.
-    Throwable cause = e.getCause();
-    if ((cause != null && cause instanceof NotServingRegionException) ||
-        (cause != null && cause instanceof RegionServerStoppedException) ||
-        e instanceof OutOfOrderScannerNextException || e instanceof UnknownScannerException ||
-        e instanceof ScannerResetException || e instanceof LeaseException) {
-      // Pass. It is easier writing the if loop test as list of what is allowed rather than
-      // as a list of what is not allowed... so if in here, it means we do not throw.
-      if (retriesLeft <= 0) {
-        throw e; // no more retries
-      }
-    } else {
-      throw e;
-    }
-
-    // Else, its signal from depths of ScannerCallable that we need to reset the scanner.
-    if (this.lastResult != null) {
-      // The region has moved. We need to open a brand new scanner at the new location.
-      // Reset the startRow to the row we've seen last so that the new scanner starts at
-      // the correct row. Otherwise we may see previously returned rows again.
-      // If the lastRow is not partial, then we should start from the next row. As now we can
-      // exclude the start row, the logic here is the same for both normal scan and reversed scan.
-      // If lastResult is partial then include it, otherwise exclude it.
-      scan.withStartRow(lastResult.getRow(), lastResult.mayHaveMoreCellsInRow());
-    }
-    if (e instanceof OutOfOrderScannerNextException) {
-      if (retryAfterOutOfOrderException.isTrue()) {
-        retryAfterOutOfOrderException.setValue(false);
-      } else {
-        // TODO: Why wrap this in a DNRIOE when it already is a DNRIOE?
-        throw new DoNotRetryIOException(
-            "Failed after retry of OutOfOrderScannerNextException: was there a rpc timeout?", e);
-      }
-    }
-    // Clear region.
-    this.currentRegion = null;
-    // Set this to zero so we don't try and do an rpc and close on remote server when
-    // the exception we got was UnknownScanner or the Server is going down.
-    callable = null;
-  }
-
-  /**
-   * Contact the servers to load more {@link Result}s in the cache.
-   */
-  protected void loadCache() throws IOException {
-    // check if scanner was closed during previous prefetch
-    if (closed) {
-      return;
-    }
-    long remainingResultSize = maxScannerResultSize;
-    int countdown = this.caching;
-    // This is possible if we just stopped at the boundary of a region in the previous call.
-    if (callable == null && !moveToNextRegion()) {
-      closed = true;
-      return;
-    }
-    // This flag is set when we want to skip the result returned. We do
-    // this when we reset scanner because it split under us.
-    MutableBoolean retryAfterOutOfOrderException = new MutableBoolean(true);
-    // Even if we are retrying due to UnknownScannerException, ScannerResetException, etc. we should
-    // make sure that we are not retrying indefinitely.
-    int retriesLeft = getRetries();
-    for (;;) {
-      Result[] values;
-      try {
-        // Server returns a null values if scanning is to stop. Else,
-        // returns an empty array if scanning is to go on and we've just
-        // exhausted current region.
-        // now we will also fetch data when openScanner, so do not make a next call again if values
-        // is already non-null.
-        values = call(callable, caller, scannerTimeout, true);
-        // When the replica switch happens, we need to do certain operations again.
-        // The callable will openScanner with the right startkey but we need to pick up
-        // from there. Bypass the rest of the loop and let the catch-up happen in the beginning
-        // of the loop as it happens for the cases where we see exceptions.
-        if (callable.switchedToADifferentReplica()) {
-          // Any accumulated partial results are no longer valid since the callable will
-          // openScanner with the correct startkey and we must pick up from there
-          scanResultCache.clear();
-          this.currentRegion = callable.getHRegionInfo();
-        }
-        retryAfterOutOfOrderException.setValue(true);
-      } catch (DoNotRetryIOException e) {
-        handleScanError(e, retryAfterOutOfOrderException, retriesLeft--);
-        // reopen the scanner
-        if (!moveToNextRegion()) {
-          break;
-        }
-        continue;
-      }
-      long currentTime = System.currentTimeMillis();
-      if (this.scanMetrics != null) {
-        this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime - lastNext);
-      }
-      lastNext = currentTime;
-      // Groom the array of Results that we received back from the server before adding that
-      // Results to the scanner's cache. If partial results are not allowed to be seen by the
-      // caller, all book keeping will be performed within this method.
-      int numberOfCompleteRowsBefore = scanResultCache.numberOfCompleteRows();
-      Result[] resultsToAddToCache =
-          scanResultCache.addAndGet(values, callable.isHeartbeatMessage());
-      int numberOfCompleteRows =
-          scanResultCache.numberOfCompleteRows() - numberOfCompleteRowsBefore;
-      for (Result rs : resultsToAddToCache) {
-        cache.add(rs);
-        long estimatedHeapSizeOfResult = calcEstimatedSize(rs);
-        countdown--;
-        remainingResultSize -= estimatedHeapSizeOfResult;
-        addEstimatedSize(estimatedHeapSizeOfResult);
-        this.lastResult = rs;
-      }
-
-      if (scan.getLimit() > 0) {
-        int newLimit = scan.getLimit() - numberOfCompleteRows;
-        assert newLimit >= 0;
-        scan.setLimit(newLimit);
-      }
-      if (scan.getLimit() == 0 || scanExhausted(values)) {
-        closeScanner();
-        closed = true;
-        break;
-      }
-      boolean regionExhausted = regionExhausted(values);
-      if (callable.isHeartbeatMessage()) {
-        if (!cache.isEmpty()) {
-          // Caller of this method just wants a Result. If we see a heartbeat message, it means
-          // processing of the scan is taking a long time server side. Rather than continue to
-          // loop until a limit (e.g. size or caching) is reached, break out early to avoid causing
-          // unnecesary delays to the caller
-          LOG.trace("Heartbeat message received and cache contains Results. " +
-            "Breaking out of scan loop");
-          // we know that the region has not been exhausted yet so just break without calling
-          // closeScannerIfExhausted
-          break;
-        }
-      }
-      if (cache.isEmpty() && !closed && scan.isNeedCursorResult()) {
-        if (callable.isHeartbeatMessage() && callable.getCursor() != null) {
-          // Use cursor row key from server
-          cache.add(Result.createCursorResult(callable.getCursor()));
-          break;
-        }
-        if (values.length > 0) {
-          // It is size limit exceed and we need return the last Result's row.
-          // When user setBatch and the scanner is reopened, the server may return Results that
-          // user has seen and the last Result can not be seen because the number is not enough.
-          // So the row keys of results may not be same, we must use the last one.
-          cache.add(Result.createCursorResult(new Cursor(values[values.length - 1].getRow())));
-          break;
-        }
-      }
-      if (countdown <= 0) {
-        // we have enough result.
-        closeScannerIfExhausted(regionExhausted);
-        break;
-      }
-      if (remainingResultSize <= 0) {
-        if (!cache.isEmpty()) {
-          closeScannerIfExhausted(regionExhausted);
-          break;
-        } else {
-          // we have reached the max result size but we still can not find anything to return to the
-          // user. Reset the maxResultSize and try again.
-          remainingResultSize = maxScannerResultSize;
-        }
-      }
-      // we are done with the current region
-      if (regionExhausted) {
-        if (!moveToNextRegion()) {
-          closed = true;
-          break;
-        }
-      }
-    }
-  }
-
-  protected void addEstimatedSize(long estimatedHeapSizeOfResult) {
-    return;
-  }
-
-  @VisibleForTesting
-  public int getCacheCount() {
-    return cache != null ? cache.size() : 0;
-  }
-
-  @Override
-  public void close() {
-    if (callable != null) {
-      callable.setClose();
-      try {
-        call(callable, caller, scannerTimeout, false);
-      } catch (UnknownScannerException e) {
-        // We used to catch this error, interpret, and rethrow. However, we
-        // have since decided that it's not nice for a scanner's close to
-        // throw exceptions. Chances are it was just due to lease time out.
-        LOG.debug("scanner failed to close", e);
-      } catch (IOException e) {
-        /* An exception other than UnknownScanner is unexpected. */
-        LOG.warn("scanner failed to close.", e);
-      }
-      callable = null;
-    }
-    closed = true;
-  }
-
-  @Override
-  public boolean renewLease() {
-    if (callable == null) {
-      return false;
-    }
-    // do not return any rows, do not advance the scanner
-    callable.setRenew(true);
-    try {
-      this.caller.callWithoutRetries(callable, this.scannerTimeout);
-      return true;
-    } catch (Exception e) {
-      LOG.debug("scanner failed to renew lease", e);
-      return false;
-    } finally {
-      callable.setRenew(false);
-    }
-  }
-
-  protected void initCache() {
-    initSyncCache();
-  }
-
-  @Override
-  public Result next() throws IOException {
-    return nextWithSyncCache();
-  }
-}
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
deleted file mode 100644
index c7006a8..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java
+++ /dev/null
@@ -1,57 +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.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-
-/**
- * A RegionServerCallable set to use the Client protocol.
- * Also includes some utility methods so can hide protobuf references here rather than have them
- * sprinkled about the code base.
- * @param <T>
- */
-@InterfaceAudience.Private
-public abstract class ClientServiceCallable<T>
-    extends RegionServerCallable<T, ClientProtos.ClientService.BlockingInterface> {
-
-  public ClientServiceCallable(ConnectionImplementation connection, TableName tableName, byte[] row,
-      RpcController rpcController, int priority) {
-    super(connection, tableName, row, rpcController, priority);
-  }
-
-  @Override
-  protected void setStubByServiceName(ServerName serviceName) throws IOException {
-    setStub(getConnection().getClient(serviceName));
-  }
-
-  // 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 {
-    return getStub().get(getRpcController(), request);
-  }
-
-  protected ClientProtos.MutateResponse doMutate(ClientProtos.MutateRequest request)
-      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/ClientSimpleScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
deleted file mode 100644
index e5d7b97..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
+++ /dev/null
@@ -1,66 +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 static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowAfter;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForScan;
-
-import java.io.IOException;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-
-/**
- * ClientSimpleScanner implements a sync scanner behaviour.
- * The cache is a simple list.
- * The prefetch is invoked only when the application finished processing the entire cache.
- */
-@InterfaceAudience.Private
-public class ClientSimpleScanner extends ClientScanner {
-  public ClientSimpleScanner(Configuration configuration, Scan scan, TableName name,
-      ConnectionImplementation connection, RpcRetryingCallerFactory rpcCallerFactory,
-      RpcControllerFactory rpcControllerFactory, ExecutorService pool,
-      int replicaCallTimeoutMicroSecondScan) throws IOException {
-    super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool,
-      replicaCallTimeoutMicroSecondScan);
-  }
-
-  @Override
-  protected boolean setNewStartKey() {
-    if (noMoreResultsForScan(scan, currentRegion)) {
-      return false;
-    }
-    scan.withStartRow(currentRegion.getEndKey(), true);
-    return true;
-  }
-
-  @Override
-  protected ScannerCallable createScannerCallable() {
-    if (!scan.includeStartRow() && !isEmptyStartRow(scan.getStartRow())) {
-      // we have not implemented locate to next row for sync client yet, so here we change the
-      // inclusive of start row to true.
-      scan.withStartRow(createClosestRowAfter(scan.getStartRow()), true);
-    }
-    return new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics,
-        this.rpcControllerFactory);
-  }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
index 53859c2..9edf8c2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
@@ -44,6 +44,7 @@ public class ConnectionConfiguration {
   public static final String PRIMARY_SCAN_TIMEOUT_MICROSECOND =
     "hbase.client.replicaCallTimeout.scan";
   public static final int PRIMARY_SCAN_TIMEOUT_MICROSECOND_DEFAULT = 1000000; // 1s
+  public static final String LOG_SCANNER_ACTIVITY = "hbase.client.log.scanner.activity";
 
   private final long writeBufferSize;
   private final long writeBufferPeriodicFlushTimeoutMs;
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 b6d0161..2b432e4 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
@@ -232,14 +232,6 @@ public class ConnectionFactory {
   }
 
   /**
-   * Create a {@link ConnectionImplementation}, internal use only.
-   */
-  static ConnectionImplementation createConnectionImpl(Configuration conf, ExecutorService pool,
-      User user) throws IOException {
-    return new ConnectionImplementation(conf, pool, user);
-  }
-
-  /**
    * Call {@link #createAsyncConnection(Configuration)} using default HBaseConfiguration.
    * @see #createAsyncConnection(Configuration)
    * @return AsyncConnection object wrapped by CompletableFuture
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
deleted file mode 100644
index edfc258..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ /dev/null
@@ -1,2185 +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 static org.apache.hadoop.hbase.client.ConnectionUtils.NO_NONCE_GENERATOR;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.getStubKey;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts;
-import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY;
-import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
-import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsentEx;
-
-import edu.umd.cs.findbugs.annotations.Nullable;
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.lang.reflect.UndeclaredThrowableException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Date;
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReentrantLock;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.AuthUtil;
-import org.apache.hadoop.hbase.CallQueueTooBigException;
-import org.apache.hadoop.hbase.ChoreService;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.MasterNotRunningException;
-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.TableNotEnabledException;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.client.Scan.ReadType;
-import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
-import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory;
-import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
-import org.apache.hadoop.hbase.exceptions.RegionMovedException;
-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.log.HBaseMarkers;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ExceptionUtil;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
-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.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasUserPermissionsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasUserPermissionsResponse;
-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.ClientProtos.ClientService.BlockingInterface;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRpcThrottleEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRpcThrottleEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchExceedThrottleQuotaRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchExceedThrottleQuotaResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchRpcThrottleRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchRpcThrottleResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
-
-/**
- * Main implementation of {@link Connection} and {@link ClusterConnection} interfaces.
- * Encapsulates connection to zookeeper and regionservers.
- */
-@edu.umd.cs.findbugs.annotations.SuppressWarnings(
-    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 Connection, Closeable {
-  public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server";
-  private static final Logger LOG = LoggerFactory.getLogger(ConnectionImplementation.class);
-
-  private static final String RESOLVE_HOSTNAME_ON_FAIL_KEY = "hbase.resolve.hostnames.on.failure";
-
-  private final boolean hostnamesCanChange;
-  private final long pause;
-  private final long pauseForCQTBE;// pause for CallQueueTooBigException, if specified
-  private boolean useMetaReplicas;
-  private final int metaReplicaCallTimeoutScanInMicroSecond;
-  private final int numTries;
-  final int rpcTimeout;
-
-  /**
-   * Global nonceGenerator shared per client.Currently there's no reason to limit its scope.
-   * Once it's set under nonceGeneratorCreateLock, it is never unset or changed.
-   */
-  private static volatile NonceGenerator nonceGenerator = null;
-  /** The nonce generator lock. Only taken when creating Connection, which gets a private copy. */
-  private static final Object nonceGeneratorCreateLock = new Object();
-
-  private final AsyncProcess asyncProcess;
-  // single tracker per connection
-  private final ServerStatisticTracker stats;
-
-  private volatile boolean closed;
-  private volatile boolean aborted;
-
-  // package protected for the tests
-  ClusterStatusListener clusterStatusListener;
-
-  private final Object metaRegionLock = new Object();
-
-  private final Object masterLock = new Object();
-
-  // thread executor shared by all Table instances created
-  // by this connection
-  private volatile ThreadPoolExecutor batchPool = null;
-  // meta thread executor shared by all Table instances created
-  // by this connection
-  private volatile ThreadPoolExecutor metaLookupPool = null;
-  private volatile boolean cleanupPool = false;
-
-  private final Configuration conf;
-
-  // cache the configuration value for tables so that we can avoid calling
-  // the expensive Configuration to fetch the value multiple times.
-  private final ConnectionConfiguration connectionConfig;
-
-  // Client rpc instance.
-  private final RpcClient rpcClient;
-
-  private final MetaCache metaCache;
-  private final MetricsConnection metrics;
-
-  protected User user;
-
-  private final RpcRetryingCallerFactory rpcCallerFactory;
-
-  private final RpcControllerFactory rpcControllerFactory;
-
-  private final RetryingCallerInterceptor interceptor;
-
-  /**
-   * Cluster registry of basic info such as clusterid and meta region location.
-   */
-  private final AsyncRegistry registry;
-
-  private final ClientBackoffPolicy backoffPolicy;
-
-  /**
-   * Allow setting an alternate BufferedMutator implementation via
-   * config. If null, use default.
-   */
-  private final String alternateBufferedMutatorClassName;
-
-  /** lock guards against multiple threads trying to query the meta region at the same time */
-  private final ReentrantLock userRegionLock = new ReentrantLock();
-
-  private ChoreService authService;
-
-  /**
-   * constructor
-   * @param conf Configuration object
-   */
-  ConnectionImplementation(Configuration conf, ExecutorService pool, User user) throws IOException {
-    this.conf = conf;
-    this.user = user;
-    if (user != null && user.isLoginFromKeytab()) {
-      spawnRenewalChore(user.getUGI());
-    }
-    this.batchPool = (ThreadPoolExecutor) pool;
-    this.connectionConfig = new ConnectionConfiguration(conf);
-    this.closed = false;
-    this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
-        HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
-    long configuredPauseForCQTBE = conf.getLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, pause);
-    if (configuredPauseForCQTBE < pause) {
-      LOG.warn("The " + HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE + " setting: "
-          + configuredPauseForCQTBE + " is smaller than " + HConstants.HBASE_CLIENT_PAUSE
-          + ", will use " + pause + " instead.");
-      this.pauseForCQTBE = pause;
-    } else {
-      this.pauseForCQTBE = configuredPauseForCQTBE;
-    }
-    this.useMetaReplicas = conf.getBoolean(HConstants.USE_META_REPLICAS,
-      HConstants.DEFAULT_USE_META_REPLICAS);
-    this.metaReplicaCallTimeoutScanInMicroSecond =
-        connectionConfig.getMetaReplicaCallTimeoutMicroSecondScan();
-
-    // how many times to try, one more than max *retry* time
-    this.numTries = retries2Attempts(connectionConfig.getRetriesNumber());
-    this.rpcTimeout = conf.getInt(
-        HConstants.HBASE_RPC_TIMEOUT_KEY,
-        HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
-    if (conf.getBoolean(NonceGenerator.CLIENT_NONCES_ENABLED_KEY, true)) {
-      synchronized (nonceGeneratorCreateLock) {
-        if (nonceGenerator == null) {
-          nonceGenerator = PerClientRandomNonceGenerator.get();
-        }
-      }
-    } else {
-      nonceGenerator = NO_NONCE_GENERATOR;
-    }
-
-    this.stats = ServerStatisticTracker.create(conf);
-    this.interceptor = (new RetryingCallerInterceptorFactory(conf)).build();
-    this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf, interceptor, this.stats);
-    this.backoffPolicy = ClientBackoffPolicyFactory.create(conf);
-    this.asyncProcess = new AsyncProcess(this, conf, rpcCallerFactory, rpcControllerFactory);
-    if (conf.getBoolean(CLIENT_SIDE_METRICS_ENABLED_KEY, false)) {
-      this.metrics =
-        new MetricsConnection(this.toString(), this::getBatchPool, this::getMetaLookupPool);
-    } else {
-      this.metrics = null;
-    }
-    this.metaCache = new MetaCache(this.metrics);
-
-    boolean shouldListen = conf.getBoolean(HConstants.STATUS_PUBLISHED,
-        HConstants.STATUS_PUBLISHED_DEFAULT);
-    this.hostnamesCanChange = conf.getBoolean(RESOLVE_HOSTNAME_ON_FAIL_KEY, true);
-    Class<? extends ClusterStatusListener.Listener> listenerClass =
-        conf.getClass(ClusterStatusListener.STATUS_LISTENER_CLASS,
-            ClusterStatusListener.DEFAULT_STATUS_LISTENER_CLASS,
-            ClusterStatusListener.Listener.class);
-
-    // Is there an alternate BufferedMutator to use?
-    this.alternateBufferedMutatorClassName =
-        this.conf.get(BufferedMutator.CLASSNAME_KEY);
-
-    try {
-      this.registry = AsyncRegistryFactory.getRegistry(conf);
-      retrieveClusterId();
-
-      this.rpcClient = RpcClientFactory.createClient(this.conf, this.clusterId, this.metrics);
-
-      // Do we publish the status?
-      if (shouldListen) {
-        if (listenerClass == null) {
-          LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but " +
-              ClusterStatusListener.STATUS_LISTENER_CLASS + " is not set - not listening status");
-        } else {
-          clusterStatusListener = new ClusterStatusListener(
-              new ClusterStatusListener.DeadServerHandler() {
-                @Override
-                public void newDead(ServerName sn) {
-                  clearCaches(sn);
-                  rpcClient.cancelConnections(sn);
-                }
-              }, conf, listenerClass);
-        }
-      }
-    } catch (Throwable e) {
-      // avoid leaks: registry, rpcClient, ...
-      LOG.debug("connection construction failed", e);
-      close();
-      throw e;
-    }
-  }
-
-  private void spawnRenewalChore(final UserGroupInformation user) {
-    authService = new ChoreService("Relogin service");
-    authService.scheduleChore(AuthUtil.getAuthRenewalChore(user));
-  }
-
-  /**
-   * @param useMetaReplicas
-   */
-  @VisibleForTesting
-  void setUseMetaReplicas(final boolean useMetaReplicas) {
-    this.useMetaReplicas = useMetaReplicas;
-  }
-
-  /**
-   * @param conn The connection for which to replace the generator.
-   * @param cnm Replaces the nonce generator used, for testing.
-   * @return old nonce generator.
-   */
-  @VisibleForTesting
-  static NonceGenerator injectNonceGeneratorForTesting(
-      ConnectionImplementation conn, NonceGenerator cnm) {
-    NonceGenerator ng = conn.getNonceGenerator();
-    LOG.warn("Nonce generator is being replaced by test code for "
-      + cnm.getClass().getName());
-    nonceGenerator = cnm;
-    return ng;
-  }
-
-  @Override
-  public Table getTable(TableName tableName) throws IOException {
-    return getTable(tableName, getBatchPool());
-  }
-
-  @Override
-  public TableBuilder getTableBuilder(TableName tableName, ExecutorService pool) {
-    return new TableBuilderBase(tableName, connectionConfig) {
-
-      @Override
-      public Table build() {
-        return new HTable(ConnectionImplementation.this, this, rpcCallerFactory,
-            rpcControllerFactory, pool);
-      }
-    };
-  }
-
-  @Override
-  public BufferedMutator getBufferedMutator(BufferedMutatorParams params) {
-    if (params.getTableName() == null) {
-      throw new IllegalArgumentException("TableName cannot be null.");
-    }
-    if (params.getPool() == null) {
-      params.pool(HTable.getDefaultExecutor(getConfiguration()));
-    }
-    if (params.getWriteBufferSize() == BufferedMutatorParams.UNSET) {
-      params.writeBufferSize(connectionConfig.getWriteBufferSize());
-    }
-    if (params.getWriteBufferPeriodicFlushTimeoutMs() == BufferedMutatorParams.UNSET) {
-      params.setWriteBufferPeriodicFlushTimeoutMs(
-              connectionConfig.getWriteBufferPeriodicFlushTimeoutMs());
-    }
-    if (params.getWriteBufferPeriodicFlushTimerTickMs() == BufferedMutatorParams.UNSET) {
-      params.setWriteBufferPeriodicFlushTimerTickMs(
-              connectionConfig.getWriteBufferPeriodicFlushTimerTickMs());
-    }
-    if (params.getMaxKeyValueSize() == BufferedMutatorParams.UNSET) {
-      params.maxKeyValueSize(connectionConfig.getMaxKeyValueSize());
-    }
-    // Look to see if an alternate BufferedMutation implementation is wanted.
-    // Look in params and in config. If null, use default.
-    String implementationClassName = params.getImplementationClassName();
-    if (implementationClassName == null) {
-      implementationClassName = this.alternateBufferedMutatorClassName;
-    }
-    if (implementationClassName == null) {
-      return new BufferedMutatorImpl(this, rpcCallerFactory, rpcControllerFactory, params);
-    }
-    try {
-      return (BufferedMutator)ReflectionUtils.newInstance(Class.forName(implementationClassName),
-          this, rpcCallerFactory, rpcControllerFactory, params);
-    } catch (ClassNotFoundException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public RegionLocator getRegionLocator(TableName tableName) throws IOException {
-    return new HRegionLocator(tableName, this);
-  }
-
-  @Override
-  public Admin getAdmin() throws IOException {
-    return new HBaseAdmin(this);
-  }
-
-  @Override
-  public Hbck getHbck() throws IOException {
-    return getHbck(get(registry.getMasterAddress()));
-  }
-
-  @Override
-  public Hbck getHbck(ServerName masterServer) throws IOException {
-    checkClosed();
-    if (isDeadServer(masterServer)) {
-      throw new RegionServerStoppedException(masterServer + " is dead.");
-    }
-    String key = getStubKey(MasterProtos.HbckService.BlockingInterface.class.getName(),
-      masterServer, this.hostnamesCanChange);
-
-    return new HBaseHbck(
-      (MasterProtos.HbckService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> {
-        BlockingRpcChannel channel =
-          this.rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout);
-        return MasterProtos.HbckService.newBlockingStub(channel);
-      }), rpcControllerFactory);
-  }
-
-  /**
-   * @return the MetricsConnection instance associated with this connection.
-   */
-  public MetricsConnection getConnectionMetrics() {
-    return this.metrics;
-  }
-
-  private ThreadPoolExecutor getBatchPool() {
-    if (batchPool == null) {
-      synchronized (this) {
-        if (batchPool == null) {
-          int threads = conf.getInt("hbase.hconnection.threads.max", 256);
-          this.batchPool = getThreadPool(threads, threads, "-shared", null);
-          this.cleanupPool = true;
-        }
-      }
-    }
-    return this.batchPool;
-  }
-
-  private ThreadPoolExecutor getThreadPool(int maxThreads, int coreThreads, String nameHint,
-      BlockingQueue<Runnable> passedWorkQueue) {
-    // shared HTable thread executor not yet initialized
-    return ConnectionUtils.getThreadPool(conf, maxThreads, coreThreads, () -> toString() + nameHint,
-      passedWorkQueue);
-  }
-
-  private ThreadPoolExecutor getMetaLookupPool() {
-    if (this.metaLookupPool == null) {
-      synchronized (this) {
-        if (this.metaLookupPool == null) {
-          //Some of the threads would be used for meta replicas
-          //To start with, threads.max.core threads can hit the meta (including replicas).
-          //After that, requests will get queued up in the passed queue, and only after
-          //the queue is full, a new thread will be started
-          int threads = conf.getInt("hbase.hconnection.meta.lookup.threads.max", 128);
-          this.metaLookupPool = getThreadPool(
-             threads,
-             threads,
-             "-metaLookup-shared-", new LinkedBlockingQueue<>());
-        }
-      }
-    }
-    return this.metaLookupPool;
-  }
-
-  protected ExecutorService getCurrentMetaLookupPool() {
-    return metaLookupPool;
-  }
-
-  protected ExecutorService getCurrentBatchPool() {
-    return batchPool;
-  }
-
-  private void shutdownPools() {
-    if (this.cleanupPool && this.batchPool != null && !this.batchPool.isShutdown()) {
-      ConnectionUtils.shutdownPool(this.batchPool);
-    }
-    if (this.metaLookupPool != null && !this.metaLookupPool.isShutdown()) {
-      ConnectionUtils.shutdownPool(this.metaLookupPool);
-    }
-  }
-
-  /**
-   * For tests only.
-   */
-  @VisibleForTesting
-  RpcClient getRpcClient() {
-    return rpcClient;
-  }
-
-  /**
-   * An identifier that will remain the same for a given connection.
-   */
-  @Override
-  public String toString(){
-    return "hconnection-0x" + Integer.toHexString(hashCode());
-  }
-
-  protected String clusterId = null;
-
-  protected void retrieveClusterId() {
-    if (clusterId != null) {
-      return;
-    }
-    try {
-      this.clusterId = this.registry.getClusterId().get();
-    } catch (InterruptedException | ExecutionException e) {
-      LOG.warn("Retrieve cluster id failed", e);
-    }
-    if (clusterId == null) {
-      clusterId = HConstants.CLUSTER_ID_DEFAULT;
-      LOG.debug("clusterid came back null, using default " + clusterId);
-    }
-  }
-
-  @Override
-  public Configuration getConfiguration() {
-    return this.conf;
-  }
-
-  private void checkClosed() throws DoNotRetryIOException {
-    if (this.closed) {
-      throw new DoNotRetryIOException(toString() + " closed");
-    }
-  }
-
-  /**
-   * @return true if the master is running, throws an exception otherwise
-   * @throws org.apache.hadoop.hbase.MasterNotRunningException - if the master is not running
-   * @deprecated this has been deprecated without a replacement
-   */
-  @Deprecated
-  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
-    // connection on a running master
-    MasterKeepAliveConnection m;
-    try {
-      m = getKeepAliveMasterService();
-    } catch (IOException e) {
-      throw new MasterNotRunningException(e);
-    }
-    m.close();
-    return true;
-  }
-
-  /**
-   * 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);
-  }
-
-  /**
-   * 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);
-  }
-
-  /**
-   * @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);
-  }
-
-  /**
-   * 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();
-    try {
-      if (!isTableEnabled(tableName)) {
-        LOG.debug("Table {} not enabled", tableName);
-        return false;
-      }
-      List<Pair<RegionInfo, ServerName>> locations =
-        MetaTableAccessor.getTableRegionsAndLocations(this, tableName, true);
-
-      int notDeployed = 0;
-      int regionCount = 0;
-      for (Pair<RegionInfo, ServerName> pair : locations) {
-        RegionInfo info = pair.getFirst();
-        if (pair.getSecond() == null) {
-          LOG.debug("Table {} has not deployed region {}", tableName,
-              pair.getFirst().getEncodedName());
-          notDeployed++;
-        } else if (splitKeys != null
-            && !Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
-          for (byte[] splitKey : splitKeys) {
-            // Just check if the splitkey is available
-            if (Bytes.equals(info.getStartKey(), splitKey)) {
-              regionCount++;
-              break;
-            }
-          }
-        } else {
-          // Always empty start row should be counted
-          regionCount++;
-        }
-      }
-      if (notDeployed > 0) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Table {} has {} regions not deployed", tableName, notDeployed);
-        }
-        return false;
-      } else if (splitKeys != null && regionCount != splitKeys.length + 1) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Table {} expected to have {} regions, but only {} available", tableName,
-              splitKeys.length + 1, regionCount);
-        }
-        return false;
-      } else {
-        LOG.trace("Table {} should be available", tableName);
-        return true;
-      }
-    } catch (TableNotFoundException tnfe) {
-      LOG.warn("Table {} does not exist", tableName);
-      return false;
-    }
-  }
-
-  private boolean isDeadServer(ServerName sn) {
-    if (clusterStatusListener == null) {
-      return false;
-    } else {
-      return clusterStatusListener.isDeadServer(sn);
-    }
-  }
-
-  /**
-   * 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);
-  }
-
-  /**
-   * 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)) {
-      regions = Collections.singletonList(RegionInfoBuilder.FIRST_META_REGIONINFO);
-    } else {
-      regions = MetaTableAccessor.getTableRegions(this, tableName, !offlined);
-    }
-    List<HRegionLocation> locations = new ArrayList<>();
-    for (RegionInfo regionInfo : regions) {
-      if (!RegionReplicaUtil.isDefaultReplica(regionInfo)) {
-        continue;
-      }
-      RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true);
-      if (list != null) {
-        for (HRegionLocation loc : list.getRegionLocations()) {
-          if (loc != null) {
-            locations.add(loc);
-          }
-        }
-      }
-    }
-    return locations;
-  }
-
-  /**
-   * 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();
-  }
-
-  /**
-   * 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);
-  }
-
-  /**
-   * 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
-    // the first time a disabled table is interacted with.
-    if (!tableName.equals(TableName.META_TABLE_NAME) && isTableDisabled(tableName)) {
-      throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled.");
-    }
-
-    return locateRegion(tableName, row, false, true, replicaId);
-  }
-
-  /**
-   * @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);
-  }
-
-  /**
-   * @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) {
-      throw new IllegalArgumentException("table name cannot be null or zero length");
-    }
-    if (tableName.equals(TableName.META_TABLE_NAME)) {
-      return locateMeta(tableName, useCache, replicaId);
-    } else {
-      // Region not in the cache - have to go to the meta RS
-      return locateRegionInMeta(tableName, row, useCache, retry, replicaId);
-    }
-  }
-
-  private RegionLocations locateMeta(final TableName tableName,
-      boolean useCache, int replicaId) throws IOException {
-    // HBASE-10785: We cache the location of the META itself, so that we are not overloading
-    // zookeeper with one request for every region lookup. We cache the META with empty row
-    // key in MetaCache.
-    byte[] metaCacheKey = HConstants.EMPTY_START_ROW; // use byte[0] as the row for meta
-    RegionLocations locations = null;
-    if (useCache) {
-      locations = getCachedLocation(tableName, metaCacheKey);
-      if (locations != null && locations.getRegionLocation(replicaId) != null) {
-        return locations;
-      }
-    }
-
-    // only one thread should do the lookup.
-    synchronized (metaRegionLock) {
-      // Check the cache again for a hit in case some other thread made the
-      // same query while we were waiting on the lock.
-      if (useCache) {
-        locations = getCachedLocation(tableName, metaCacheKey);
-        if (locations != null && locations.getRegionLocation(replicaId) != null) {
-          return locations;
-        }
-      }
-
-      // Look up from zookeeper
-      locations = get(this.registry.getMetaRegionLocation());
-      if (locations != null) {
-        cacheLocation(tableName, locations);
-      }
-    }
-    return locations;
-  }
-
-  /**
-   * Search the hbase:meta table for the HRegionLocation info that contains the table and row we're
-   * seeking.
-   */
-  private RegionLocations locateRegionInMeta(TableName tableName, byte[] row, boolean useCache,
-      boolean retry, int replicaId) throws IOException {
-    // If we are supposed to be using the cache, look in the cache to see if we already have the
-    // region.
-    if (useCache) {
-      RegionLocations locations = getCachedLocation(tableName, row);
-      if (locations != null && locations.getRegionLocation(replicaId) != null) {
-        return locations;
-      }
-    }
-    // build the key of the meta region we should be looking for.
-    // the extra 9's on the end are necessary to allow "exact" matches
-    // without knowing the precise region names.
-    byte[] metaStartKey = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
-    byte[] metaStopKey =
-      RegionInfo.createRegionName(tableName, HConstants.EMPTY_START_ROW, "", false);
-    Scan s = new Scan().withStartRow(metaStartKey).withStopRow(metaStopKey, true)
-      .addFamily(HConstants.CATALOG_FAMILY).setReversed(true).setCaching(5)
-      .setReadType(ReadType.PREAD);
-    if (this.useMetaReplicas) {
-      s.setConsistency(Consistency.TIMELINE);
-    }
-    int maxAttempts = (retry ? numTries : 1);
-    boolean relocateMeta = false;
-    for (int tries = 0; ; tries++) {
-      if (tries >= maxAttempts) {
-        throw new NoServerForRegionException("Unable to find region for "
-            + Bytes.toStringBinary(row) + " in " + tableName + " after " + tries + " tries.");
-      }
-      if (useCache) {
-        RegionLocations locations = getCachedLocation(tableName, row);
-        if (locations != null && locations.getRegionLocation(replicaId) != null) {
-          return locations;
-        }
-      } else {
-        // If we are not supposed to be using the cache, delete any existing cached location
-        // so it won't interfere.
-        // We are only supposed to clean the cache for the specific replicaId
-        metaCache.clearCache(tableName, row, replicaId);
-      }
-      // Query the meta region
-      long pauseBase = this.pause;
-      userRegionLock.lock();
-      try {
-        if (useCache) {// re-check cache after get lock
-          RegionLocations locations = getCachedLocation(tableName, row);
-          if (locations != null && locations.getRegionLocation(replicaId) != null) {
-            return locations;
-          }
-        }
-        if (relocateMeta) {
-          relocateRegion(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW,
-            RegionInfo.DEFAULT_REPLICA_ID);
-        }
-        s.resetMvccReadPoint();
-        try (ReversedClientScanner rcs =
-          new ReversedClientScanner(conf, s, TableName.META_TABLE_NAME, this, rpcCallerFactory,
-            rpcControllerFactory, getMetaLookupPool(), metaReplicaCallTimeoutScanInMicroSecond)) {
-          boolean tableNotFound = true;
-          for (;;) {
-            Result regionInfoRow = rcs.next();
-            if (regionInfoRow == null) {
-              if (tableNotFound) {
-                throw new TableNotFoundException(tableName);
-              } else {
-                throw new IOException(
-                  "Unable to find region for " + Bytes.toStringBinary(row) + " in " + tableName);
-              }
-            }
-            tableNotFound = false;
-            // convert the row result into the HRegionLocation we need!
-            RegionLocations locations = MetaTableAccessor.getRegionLocations(regionInfoRow);
-            if (locations == null || locations.getRegionLocation(replicaId) == null) {
-              throw new IOException("RegionInfo null in " + tableName + ", row=" + regionInfoRow);
-            }
-            RegionInfo regionInfo = locations.getRegionLocation(replicaId).getRegion();
-            if (regionInfo == null) {
-              throw new IOException("RegionInfo null or empty in " + TableName.META_TABLE_NAME +
-                ", row=" + regionInfoRow);
-            }
-            // See HBASE-20182. It is possible that we locate to a split parent even after the
-            // children are online, so here we need to skip this region and go to the next one.
-            if (regionInfo.isSplitParent()) {
-              continue;
-            }
-            if (regionInfo.isOffline()) {
-              throw new RegionOfflineException("Region offline; disable table call? " +
-                  regionInfo.getRegionNameAsString());
-            }
-            // It is possible that the split children have not been online yet and we have skipped
-            // the parent in the above condition, so we may have already reached a region which does
-            // not contains us.
-            if (!regionInfo.containsRow(row)) {
-              throw new IOException(
-                "Unable to find region for " + Bytes.toStringBinary(row) + " in " + tableName);
-            }
-            ServerName serverName = locations.getRegionLocation(replicaId).getServerName();
-            if (serverName == null) {
-              throw new NoServerForRegionException("No server address listed in " +
-                TableName.META_TABLE_NAME + " for region " + regionInfo.getRegionNameAsString() +
-                " containing row " + Bytes.toStringBinary(row));
-            }
-            if (isDeadServer(serverName)) {
-              throw new RegionServerStoppedException(
-                "hbase:meta says the region " + regionInfo.getRegionNameAsString() +
-                  " is managed by the server " + serverName + ", but it is dead.");
-            }
-            // Instantiate the location
-            cacheLocation(tableName, locations);
-            return locations;
-          }
-        }
-      } catch (TableNotFoundException e) {
-        // if we got this error, probably means the table just plain doesn't
-        // exist. rethrow the error immediately. this should always be coming
-        // from the HTable constructor.
-        throw e;
-      } catch (IOException e) {
-        ExceptionUtil.rethrowIfInterrupt(e);
-        if (e instanceof RemoteException) {
-          e = ((RemoteException)e).unwrapRemoteException();
-        }
-        if (e instanceof CallQueueTooBigException) {
-          // Give a special check on CallQueueTooBigException, see #HBASE-17114
-          pauseBase = this.pauseForCQTBE;
-        }
-        if (tries < maxAttempts - 1) {
-          LOG.debug("locateRegionInMeta parentTable='{}', attempt={} of {} failed; retrying " +
-            "after sleep of {}", TableName.META_TABLE_NAME, tries, maxAttempts, maxAttempts, e);
-        } else {
-          throw e;
-        }
-        // Only relocate the parent region if necessary
-        relocateMeta =
-          !(e instanceof RegionOfflineException || e instanceof NoServerForRegionException);
-      } finally {
-        userRegionLock.unlock();
-      }
-      try{
-        Thread.sleep(ConnectionUtils.getPauseTime(pauseBase, tries));
-      } catch (InterruptedException e) {
-        throw new InterruptedIOException("Giving up trying to location region in " +
-          "meta: thread is interrupted.");
-      }
-    }
-  }
-
-  /**
-   * Put a newly discovered HRegionLocation into the cache.
-   * @param tableName The table name.
-   * @param location the new location
-   */
-  void cacheLocation(final TableName tableName, final RegionLocations location) {
-    metaCache.cacheLocation(tableName, location);
-  }
-
-  /**
-   * Search the cache for a location that fits our table and row key.
-   * Return null if no suitable region is located.
-   * @return Null or region location found in cache.
-   */
-  RegionLocations getCachedLocation(final TableName tableName,
-      final byte [] row) {
-    return metaCache.getCachedLocation(tableName, row);
-  }
-
-  void clearRegionCache(final TableName tableName, byte[] row) {
-    metaCache.clearCache(tableName, row);
-  }
-
-  /**
-   * Clear any caches that pertain to server name <code>sn</code>.
-   * @param sn A server name
-   */
-  void clearCaches(final ServerName serverName) {
-    metaCache.clearCache(serverName);
-  }
-
-
-  /**
-   * Allows flushing the region cache.
-   */
-  @Override
-  public void clearRegionLocationCache() {
-    metaCache.clearCache();
-  }
-
-  /**
-   * 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);
-  }
-
-  /**
-   * Put a newly discovered HRegionLocation into the cache.
-   * @param tableName The table name.
-   * @param source the source of the new location, if it's not coming from meta
-   * @param location the new location
-   */
-  private void cacheLocation(final TableName tableName, final ServerName source,
-      final HRegionLocation location) {
-    metaCache.cacheLocation(tableName, source, location);
-  }
-
-  // Map keyed by service name + regionserver to service stub implementation
-  private final ConcurrentMap<String, Object> stubs = new ConcurrentHashMap<>();
-
-  /**
-   * State of the MasterService connection/setup.
-   */
-  static class MasterServiceState {
-    Connection connection;
-
-    MasterProtos.MasterService.BlockingInterface stub;
-    int userCount;
-
-    MasterServiceState(final Connection connection) {
-      super();
-      this.connection = connection;
-    }
-
-    @Override
-    public String toString() {
-      return "MasterService";
-    }
-
-    Object getStub() {
-      return this.stub;
-    }
-
-    void clearStub() {
-      this.stub = null;
-    }
-
-    boolean isMasterRunning() throws IOException {
-      MasterProtos.IsMasterRunningResponse response = null;
-      try {
-        response = this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
-      } catch (Exception e) {
-        throw ProtobufUtil.handleRemoteException(e);
-      }
-      return response != null? response.getIsMasterRunning(): false;
-    }
-  }
-
-  /**
-   * The record of errors for servers.
-   */
-  static class ServerErrorTracker {
-    // We need a concurrent map here, as we could have multiple threads updating it in parallel.
-    private final ConcurrentMap<ServerName, ServerErrors> errorsByServer = new ConcurrentHashMap<>();
-    private final long canRetryUntil;
-    private final int maxTries;// max number to try
-    private final long startTrackingTime;
-
-    /**
-     * Constructor
-     * @param timeout how long to wait before timeout, in unit of millisecond
-     * @param maxTries how many times to try
-     */
-    public ServerErrorTracker(long timeout, int maxTries) {
-      this.maxTries = maxTries;
-      this.canRetryUntil = EnvironmentEdgeManager.currentTime() + timeout;
-      this.startTrackingTime = new Date().getTime();
-    }
-
-    /**
-     * We stop to retry when we have exhausted BOTH the number of tries and the time allocated.
-     * @param numAttempt how many times we have tried by now
-     */
-    boolean canTryMore(int numAttempt) {
-      // If there is a single try we must not take into account the time.
-      return numAttempt < maxTries || (maxTries > 1 &&
-          EnvironmentEdgeManager.currentTime() < this.canRetryUntil);
-    }
-
-    /**
-     * Calculates the back-off time for a retrying request to a particular server.
-     *
-     * @param server    The server in question.
-     * @param basePause The default hci pause.
-     * @return The time to wait before sending next request.
-     */
-    long calculateBackoffTime(ServerName server, long basePause) {
-      long result;
-      ServerErrors errorStats = errorsByServer.get(server);
-      if (errorStats != null) {
-        result = ConnectionUtils.getPauseTime(basePause, Math.max(0, errorStats.getCount() - 1));
-      } else {
-        result = 0; // yes, if the server is not in our list we don't wait before retrying.
-      }
-      return result;
-    }
-
-    /**
-     * Reports that there was an error on the server to do whatever bean-counting necessary.
-     * @param server The server in question.
-     */
-    void reportServerError(ServerName server) {
-      computeIfAbsent(errorsByServer, server, ServerErrors::new).addError();
-    }
-
-    long getStartTrackingTime() {
-      return startTrackingTime;
-    }
-
-    /**
-     * The record of errors for a server.
-     */
-    private static class ServerErrors {
-      private final AtomicInteger retries = new AtomicInteger(0);
-
-      public int getCount() {
-        return retries.get();
-      }
-
-      public void addError() {
-        retries.incrementAndGet();
-      }
-    }
-  }
-
-  /**
-   * Class to make a MasterServiceStubMaker stub.
-   */
-  private final class MasterServiceStubMaker {
-
-    private void isMasterRunning(MasterProtos.MasterService.BlockingInterface stub)
-        throws IOException {
-      try {
-        stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
-      } catch (ServiceException e) {
-        throw ProtobufUtil.handleRemoteException(e);
-      }
-    }
-
-    /**
-     * Create a stub. Try once only. It is not typed because there is no common type to protobuf
-     * services nor their interfaces. Let the caller do appropriate casting.
-     * @return A stub for master services.
-     */
-    private MasterProtos.MasterService.BlockingInterface makeStubNoRetries()
-        throws IOException, KeeperException {
-      ServerName sn = get(registry.getMasterAddress());
-      if (sn == null) {
-        String msg = "ZooKeeper available but no active master location found";
-        LOG.info(msg);
-        throw new MasterNotRunningException(msg);
-      }
-      if (isDeadServer(sn)) {
-        throw new MasterNotRunningException(sn + " is dead.");
-      }
-      // Use the security info interface name as our stub key
-      String key =
-          getStubKey(MasterProtos.MasterService.getDescriptor().getName(), sn, hostnamesCanChange);
-      MasterProtos.MasterService.BlockingInterface stub =
-          (MasterProtos.MasterService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> {
-            BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout);
-            return MasterProtos.MasterService.newBlockingStub(channel);
-          });
-      isMasterRunning(stub);
-      return stub;
-    }
-
-    /**
-     * Create a stub against the master. Retry if necessary.
-     * @return A stub to do <code>intf</code> against the master
-     * @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
-     */
-    MasterProtos.MasterService.BlockingInterface makeStub() throws IOException {
-      // The lock must be at the beginning to prevent multiple master creations
-      // (and leaks) in a multithread context
-      synchronized (masterLock) {
-        Exception exceptionCaught = null;
-        if (!closed) {
-          try {
-            return makeStubNoRetries();
-          } catch (IOException e) {
-            exceptionCaught = e;
-          } catch (KeeperException e) {
-            exceptionCaught = e;
-          }
-          throw new MasterNotRunningException(exceptionCaught);
-        } else {
-          throw new DoNotRetryIOException("Connection was closed while trying to get master");
-        }
-      }
-    }
-  }
-
-  /**
-   * Get the admin service for master.
-   */
-  public AdminProtos.AdminService.BlockingInterface getAdminForMaster() throws IOException {
-    return getAdmin(get(registry.getMasterAddress()));
-  }
-
-  /**
-   * 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();
-    if (isDeadServer(serverName)) {
-      throw new RegionServerStoppedException(serverName + " is dead.");
-    }
-    String key = getStubKey(AdminProtos.AdminService.BlockingInterface.class.getName(), serverName,
-      this.hostnamesCanChange);
-    return (AdminProtos.AdminService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> {
-      BlockingRpcChannel channel =
-          this.rpcClient.createBlockingRpcChannel(serverName, user, rpcTimeout);
-      return AdminProtos.AdminService.newBlockingStub(channel);
-    });
-  }
-
-  /**
-   * 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)) {
-      throw new RegionServerStoppedException(serverName + " is dead.");
-    }
-    String key = getStubKey(ClientProtos.ClientService.BlockingInterface.class.getName(),
-      serverName, this.hostnamesCanChange);
-    return (ClientProtos.ClientService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> {
-      BlockingRpcChannel channel =
-        this.rpcClient.createBlockingRpcChannel(serverName, user, rpcTimeout);
-      return ClientProtos.ClientService.newBlockingStub(channel);
-    });
-  }
-
-  final MasterServiceState masterServiceState = new MasterServiceState(this);
-
-  public MasterKeepAliveConnection getMaster() throws IOException {
-    return getKeepAliveMasterService();
-  }
-
-  private void resetMasterServiceState(final MasterServiceState mss) {
-    mss.userCount++;
-  }
-
-  private MasterKeepAliveConnection getKeepAliveMasterService() throws IOException {
-    synchronized (masterLock) {
-      if (!isKeepAliveMasterConnectedAndRunning(this.masterServiceState)) {
-        MasterServiceStubMaker stubMaker = new MasterServiceStubMaker();
-        this.masterServiceState.stub = stubMaker.makeStub();
-      }
-      resetMasterServiceState(this.masterServiceState);
-    }
-    // Ugly delegation just so we can add in a Close method.
-    final MasterProtos.MasterService.BlockingInterface stub = this.masterServiceState.stub;
-    return new MasterKeepAliveConnection() {
-      MasterServiceState mss = masterServiceState;
-
-      @Override
-      public MasterProtos.AbortProcedureResponse abortProcedure(
-          RpcController controller,
-          MasterProtos.AbortProcedureRequest request) throws ServiceException {
-        return stub.abortProcedure(controller, request);
-      }
-
-      @Override
-      public MasterProtos.GetProceduresResponse getProcedures(
-          RpcController controller,
-          MasterProtos.GetProceduresRequest request) throws ServiceException {
-        return stub.getProcedures(controller, request);
-      }
-
-      @Override
-      public MasterProtos.GetLocksResponse getLocks(
-          RpcController controller,
-          MasterProtos.GetLocksRequest request) throws ServiceException {
-        return stub.getLocks(controller, request);
-      }
-
-      @Override
-      public MasterProtos.AddColumnResponse addColumn(
-          RpcController controller,
-          MasterProtos.AddColumnRequest request) throws ServiceException {
-        return stub.addColumn(controller, request);
-      }
-
-      @Override
-      public MasterProtos.DeleteColumnResponse deleteColumn(RpcController controller,
-          MasterProtos.DeleteColumnRequest request)
-      throws ServiceException {
-        return stub.deleteColumn(controller, request);
-      }
-
-      @Override
-      public MasterProtos.ModifyColumnResponse modifyColumn(RpcController controller,
-          MasterProtos.ModifyColumnRequest request)
-      throws ServiceException {
-        return stub.modifyColumn(controller, request);
-      }
-
-      @Override
-      public MasterProtos.MoveRegionResponse moveRegion(RpcController controller,
-          MasterProtos.MoveRegionRequest request) throws ServiceException {
-        return stub.moveRegion(controller, request);
-      }
-
-      @Override
-      public MasterProtos.MergeTableRegionsResponse mergeTableRegions(
-          RpcController controller, MasterProtos.MergeTableRegionsRequest request)
-          throws ServiceException {
-        return stub.mergeTableRegions(controller, request);
-      }
-
-      @Override
-      public MasterProtos.AssignRegionResponse assignRegion(RpcController controller,
-          MasterProtos.AssignRegionRequest request) throws ServiceException {
-        return stub.assignRegion(controller, request);
-      }
-
-      @Override
-      public MasterProtos.UnassignRegionResponse unassignRegion(RpcController controller,
-          MasterProtos.UnassignRegionRequest request) throws ServiceException {
-        return stub.unassignRegion(controller, request);
-      }
-
-      @Override
-      public MasterProtos.OfflineRegionResponse offlineRegion(RpcController controller,
-          MasterProtos.OfflineRegionRequest request) throws ServiceException {
-        return stub.offlineRegion(controller, request);
-      }
-
-      @Override
-      public MasterProtos.SplitTableRegionResponse splitRegion(RpcController controller,
-          MasterProtos.SplitTableRegionRequest request) throws ServiceException {
-        return stub.splitRegion(controller, request);
-      }
-
-      @Override
-      public MasterProtos.DeleteTableResponse deleteTable(RpcController controller,
-          MasterProtos.DeleteTableRequest request) throws ServiceException {
-        return stub.deleteTable(controller, request);
-      }
-
-      @Override
-      public MasterProtos.TruncateTableResponse truncateTable(RpcController controller,
-          MasterProtos.TruncateTableRequest request) throws ServiceException {
-        return stub.truncateTable(controller, request);
-      }
-
-      @Override
-      public MasterProtos.EnableTableResponse enableTable(RpcController controller,
-          MasterProtos.EnableTableRequest request) throws ServiceException {
-        return stub.enableTable(controller, request);
-      }
-
-      @Override
-      public MasterProtos.DisableTableResponse disableTable(RpcController controller,
-          MasterProtos.DisableTableRequest request) throws ServiceException {
-        return stub.disableTable(controller, request);
-      }
-
-      @Override
-      public MasterProtos.ModifyTableResponse modifyTable(RpcController controller,
-          MasterProtos.ModifyTableRequest request) throws ServiceException {
-        return stub.modifyTable(controller, request);
-      }
-
-      @Override
-      public MasterProtos.CreateTableResponse createTable(RpcController controller,
-          MasterProtos.CreateTableRequest request) throws ServiceException {
-        return stub.createTable(controller, request);
-      }
-
-      @Override
-      public MasterProtos.ShutdownResponse shutdown(RpcController controller,
-          MasterProtos.ShutdownRequest request) throws ServiceException {
-        return stub.shutdown(controller, request);
-      }
-
-      @Override
-      public MasterProtos.StopMasterResponse stopMaster(RpcController controller,
-          MasterProtos.StopMasterRequest request) throws ServiceException {
-        return stub.stopMaster(controller, request);
-      }
-
-      @Override
-      public MasterProtos.IsInMaintenanceModeResponse isMasterInMaintenanceMode(
-          final RpcController controller,
-          final MasterProtos.IsInMaintenanceModeRequest request) throws ServiceException {
-        return stub.isMasterInMaintenanceMode(controller, request);
-      }
-
-      @Override
-      public MasterProtos.BalanceResponse balance(RpcController controller,
-          MasterProtos.BalanceRequest request) throws ServiceException {
-        return stub.balance(controller, request);
-      }
-
-      @Override
-      public MasterProtos.SetBalancerRunningResponse setBalancerRunning(
-          RpcController controller, MasterProtos.SetBalancerRunningRequest request)
-          throws ServiceException {
-        return stub.setBalancerRunning(controller, request);
-      }
-
-      @Override
-      public NormalizeResponse normalize(RpcController controller,
-          NormalizeRequest request) throws ServiceException {
-        return stub.normalize(controller, request);
-      }
-
-      @Override
-      public SetNormalizerRunningResponse setNormalizerRunning(
-          RpcController controller, SetNormalizerRunningRequest request)
-          throws ServiceException {
-        return stub.setNormalizerRunning(controller, request);
-      }
-
-      @Override
-      public MasterProtos.RunCatalogScanResponse runCatalogScan(RpcController controller,
-          MasterProtos.RunCatalogScanRequest request) throws ServiceException {
-        return stub.runCatalogScan(controller, request);
-      }
-
-      @Override
-      public MasterProtos.EnableCatalogJanitorResponse enableCatalogJanitor(
-          RpcController controller, MasterProtos.EnableCatalogJanitorRequest request)
-          throws ServiceException {
-        return stub.enableCatalogJanitor(controller, request);
-      }
-
-      @Override
-      public MasterProtos.IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(
-          RpcController controller, MasterProtos.IsCatalogJanitorEnabledRequest request)
-          throws ServiceException {
-        return stub.isCatalogJanitorEnabled(controller, request);
-      }
-
-      @Override
-      public MasterProtos.RunCleanerChoreResponse runCleanerChore(RpcController controller,
-          MasterProtos.RunCleanerChoreRequest request)
-          throws ServiceException {
-        return stub.runCleanerChore(controller, request);
-      }
-
-      @Override
-      public MasterProtos.SetCleanerChoreRunningResponse setCleanerChoreRunning(
-          RpcController controller, MasterProtos.SetCleanerChoreRunningRequest request)
-          throws ServiceException {
-        return stub.setCleanerChoreRunning(controller, request);
-      }
-
-      @Override
-      public MasterProtos.IsCleanerChoreEnabledResponse isCleanerChoreEnabled(
-          RpcController controller, MasterProtos.IsCleanerChoreEnabledRequest request)
-          throws ServiceException {
-        return stub.isCleanerChoreEnabled(controller, request);
-      }
-
-      @Override
-      public ClientProtos.CoprocessorServiceResponse execMasterService(
-          RpcController controller, ClientProtos.CoprocessorServiceRequest request)
-          throws ServiceException {
-        return stub.execMasterService(controller, request);
-      }
-
-      @Override
-      public MasterProtos.SnapshotResponse snapshot(RpcController controller,
-          MasterProtos.SnapshotRequest request) throws ServiceException {
-        return stub.snapshot(controller, request);
-      }
-
-      @Override
-      public MasterProtos.GetCompletedSnapshotsResponse getCompletedSnapshots(
-          RpcController controller, MasterProtos.GetCompletedSnapshotsRequest request)
-          throws ServiceException {
-        return stub.getCompletedSnapshots(controller, request);
-      }
-
-      @Override
-      public MasterProtos.DeleteSnapshotResponse deleteSnapshot(RpcController controller,
-          MasterProtos.DeleteSnapshotRequest request) throws ServiceException {
-        return stub.deleteSnapshot(controller, request);
-      }
-
-      @Override
-      public MasterProtos.IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
-          MasterProtos.IsSnapshotDoneRequest request) throws ServiceException {
-        return stub.isSnapshotDone(controller, request);
-      }
-
-      @Override
-      public MasterProtos.RestoreSnapshotResponse restoreSnapshot(
-          RpcController controller, MasterProtos.RestoreSnapshotRequest request)
-          throws ServiceException {
-        return stub.restoreSnapshot(controller, request);
-      }
-
-      @Override
-      public MasterProtos.ExecProcedureResponse execProcedure(
-          RpcController controller, MasterProtos.ExecProcedureRequest request)
-          throws ServiceException {
-        return stub.execProcedure(controller, request);
-      }
-
-      @Override
-      public MasterProtos.ExecProcedureResponse execProcedureWithRet(
-          RpcController controller, MasterProtos.ExecProcedureRequest request)
-          throws ServiceException {
-        return stub.execProcedureWithRet(controller, request);
-      }
-
-      @Override
-      public MasterProtos.IsProcedureDoneResponse isProcedureDone(RpcController controller,
-          MasterProtos.IsProcedureDoneRequest request) throws ServiceException {
-        return stub.isProcedureDone(controller, request);
-      }
-
-      @Override
-      public MasterProtos.GetProcedureResultResponse getProcedureResult(RpcController controller,
-          MasterProtos.GetProcedureResultRequest request) throws ServiceException {
-        return stub.getProcedureResult(controller, request);
-      }
-
-      @Override
-      public MasterProtos.IsMasterRunningResponse isMasterRunning(
-          RpcController controller, MasterProtos.IsMasterRunningRequest request)
-          throws ServiceException {
-        return stub.isMasterRunning(controller, request);
-      }
-
-      @Override
-      public MasterProtos.ModifyNamespaceResponse modifyNamespace(RpcController controller,
-          MasterProtos.ModifyNamespaceRequest request)
-      throws ServiceException {
-        return stub.modifyNamespace(controller, request);
-      }
-
-      @Override
-      public MasterProtos.CreateNamespaceResponse createNamespace(
-          RpcController controller,
-          MasterProtos.CreateNamespaceRequest request) throws ServiceException {
-        return stub.createNamespace(controller, request);
-      }
-
-      @Override
-      public MasterProtos.DeleteNamespaceResponse deleteNamespace(
-          RpcController controller,
-          MasterProtos.DeleteNamespaceRequest request) throws ServiceException {
-        return stub.deleteNamespace(controller, request);
-      }
-
-      @Override
-      public MasterProtos.ListNamespacesResponse listNamespaces(
-          RpcController controller,
-          MasterProtos.ListNamespacesRequest request) throws ServiceException {
-        return stub.listNamespaces(controller, request);
-      }
-
-      @Override
-      public MasterProtos.GetNamespaceDescriptorResponse getNamespaceDescriptor(
-          RpcController controller,
-          MasterProtos.GetNamespaceDescriptorRequest request) throws ServiceException {
-        return stub.getNamespaceDescriptor(controller, request);
-      }
-
-      @Override
-      public MasterProtos.ListNamespaceDescriptorsResponse listNamespaceDescriptors(
-          RpcController controller,
-          MasterProtos.ListNamespaceDescriptorsRequest request) throws ServiceException {
-        return stub.listNamespaceDescriptors(controller, request);
-      }
-
-      @Override
-      public MasterProtos.ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(
-          RpcController controller, MasterProtos.ListTableDescriptorsByNamespaceRequest request)
-              throws ServiceException {
-        return stub.listTableDescriptorsByNamespace(controller, request);
-      }
-
-      @Override
-      public MasterProtos.ListTableNamesByNamespaceResponse listTableNamesByNamespace(
-          RpcController controller, MasterProtos.ListTableNamesByNamespaceRequest request)
-              throws ServiceException {
-        return stub.listTableNamesByNamespace(controller, request);
-      }
-
-      @Override
-      public MasterProtos.GetTableStateResponse getTableState(
-              RpcController controller, MasterProtos.GetTableStateRequest request)
-              throws ServiceException {
-        return stub.getTableState(controller, request);
-      }
-
-      @Override
-      public void close() {
-        release(this.mss);
-      }
-
-      @Override
-      public MasterProtos.GetSchemaAlterStatusResponse getSchemaAlterStatus(
-          RpcController controller, MasterProtos.GetSchemaAlterStatusRequest request)
-          throws ServiceException {
-        return stub.getSchemaAlterStatus(controller, request);
-      }
-
-      @Override
-      public MasterProtos.GetTableDescriptorsResponse getTableDescriptors(
-          RpcController controller, MasterProtos.GetTableDescriptorsRequest request)
-          throws ServiceException {
-        return stub.getTableDescriptors(controller, request);
-      }
-
-      @Override
-      public MasterProtos.GetTableNamesResponse getTableNames(
-          RpcController controller, MasterProtos.GetTableNamesRequest request)
-          throws ServiceException {
-        return stub.getTableNames(controller, request);
-      }
-
-      @Override
-      public MasterProtos.GetClusterStatusResponse getClusterStatus(
-          RpcController controller, MasterProtos.GetClusterStatusRequest request)
-          throws ServiceException {
-        return stub.getClusterStatus(controller, request);
-      }
-
-      @Override
-      public MasterProtos.SetQuotaResponse setQuota(
-          RpcController controller, MasterProtos.SetQuotaRequest request)
-          throws ServiceException {
-        return stub.setQuota(controller, request);
-      }
-
-      @Override
-      public MasterProtos.MajorCompactionTimestampResponse getLastMajorCompactionTimestamp(
-          RpcController controller, MasterProtos.MajorCompactionTimestampRequest request)
-          throws ServiceException {
-        return stub.getLastMajorCompactionTimestamp(controller, request);
-      }
-
-      @Override
-      public MasterProtos.MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion(
-          RpcController controller, MasterProtos.MajorCompactionTimestampForRegionRequest request)
-          throws ServiceException {
-        return stub.getLastMajorCompactionTimestampForRegion(controller, request);
-      }
-
-      @Override
-      public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller,
-          IsBalancerEnabledRequest request) throws ServiceException {
-        return stub.isBalancerEnabled(controller, request);
-      }
-
-      @Override
-      public MasterProtos.SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(
-        RpcController controller, MasterProtos.SetSplitOrMergeEnabledRequest request)
-        throws ServiceException {
-        return stub.setSplitOrMergeEnabled(controller, request);
-      }
-
-      @Override
-      public MasterProtos.IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(
-        RpcController controller, MasterProtos.IsSplitOrMergeEnabledRequest request)
-              throws ServiceException {
-        return stub.isSplitOrMergeEnabled(controller, request);
-      }
-
-      @Override
-      public IsNormalizerEnabledResponse isNormalizerEnabled(RpcController controller,
-          IsNormalizerEnabledRequest request) throws ServiceException {
-        return stub.isNormalizerEnabled(controller, request);
-      }
-
-      @Override
-      public SecurityCapabilitiesResponse getSecurityCapabilities(RpcController controller,
-          SecurityCapabilitiesRequest request) throws ServiceException {
-        return stub.getSecurityCapabilities(controller, request);
-      }
-
-      @Override
-      public AddReplicationPeerResponse addReplicationPeer(RpcController controller,
-          AddReplicationPeerRequest request) throws ServiceException {
-        return stub.addReplicationPeer(controller, request);
-      }
-
-      @Override
-      public RemoveReplicationPeerResponse removeReplicationPeer(RpcController controller,
-          RemoveReplicationPeerRequest request) throws ServiceException {
-        return stub.removeReplicationPeer(controller, request);
-      }
-
-      @Override
-      public EnableReplicationPeerResponse enableReplicationPeer(RpcController controller,
-          EnableReplicationPeerRequest request) throws ServiceException {
-        return stub.enableReplicationPeer(controller, request);
-      }
-
-      @Override
-      public DisableReplicationPeerResponse disableReplicationPeer(RpcController controller,
-          DisableReplicationPeerRequest request) throws ServiceException {
-        return stub.disableReplicationPeer(controller, request);
-      }
-
-      @Override
-      public ListDecommissionedRegionServersResponse listDecommissionedRegionServers(RpcController controller,
-          ListDecommissionedRegionServersRequest request) throws ServiceException {
-        return stub.listDecommissionedRegionServers(controller, request);
-      }
-
-      @Override
-      public DecommissionRegionServersResponse decommissionRegionServers(RpcController controller,
-          DecommissionRegionServersRequest request) throws ServiceException {
-        return stub.decommissionRegionServers(controller, request);
-      }
-
-      @Override
-      public RecommissionRegionServerResponse recommissionRegionServer(
-          RpcController controller, RecommissionRegionServerRequest request)
-          throws ServiceException {
-        return stub.recommissionRegionServer(controller, request);
-      }
-
-      @Override
-      public GetReplicationPeerConfigResponse getReplicationPeerConfig(RpcController controller,
-          GetReplicationPeerConfigRequest request) throws ServiceException {
-        return stub.getReplicationPeerConfig(controller, request);
-      }
-
-      @Override
-      public UpdateReplicationPeerConfigResponse updateReplicationPeerConfig(
-          RpcController controller, UpdateReplicationPeerConfigRequest request)
-          throws ServiceException {
-        return stub.updateReplicationPeerConfig(controller, request);
-      }
-
-      @Override
-      public ListReplicationPeersResponse listReplicationPeers(RpcController controller,
-          ListReplicationPeersRequest request) throws ServiceException {
-        return stub.listReplicationPeers(controller, request);
-      }
-
-      @Override
-      public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(
-          RpcController controller, GetSpaceQuotaRegionSizesRequest request)
-          throws ServiceException {
-        return stub.getSpaceQuotaRegionSizes(controller, request);
-      }
-
-      @Override
-      public GetQuotaStatesResponse getQuotaStates(
-          RpcController controller, GetQuotaStatesRequest request) throws ServiceException {
-        return stub.getQuotaStates(controller, request);
-      }
-
-      @Override
-      public MasterProtos.ClearDeadServersResponse clearDeadServers(RpcController controller,
-          MasterProtos.ClearDeadServersRequest request) throws ServiceException {
-        return stub.clearDeadServers(controller, request);
-      }
-
-      @Override
-      public TransitReplicationPeerSyncReplicationStateResponse
-        transitReplicationPeerSyncReplicationState(RpcController controller,
-          TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
-        return stub.transitReplicationPeerSyncReplicationState(controller, request);
-      }
-
-      @Override
-      public SwitchRpcThrottleResponse switchRpcThrottle(RpcController controller,
-          SwitchRpcThrottleRequest request) throws ServiceException {
-        return stub.switchRpcThrottle(controller, request);
-      }
-
-      @Override
-      public IsRpcThrottleEnabledResponse isRpcThrottleEnabled(RpcController controller,
-          IsRpcThrottleEnabledRequest request) throws ServiceException {
-        return stub.isRpcThrottleEnabled(controller, request);
-      }
-
-      @Override
-      public SwitchExceedThrottleQuotaResponse switchExceedThrottleQuota(RpcController controller,
-          SwitchExceedThrottleQuotaRequest request) throws ServiceException {
-        return stub.switchExceedThrottleQuota(controller, request);
-      }
-
-      @Override
-      public AccessControlProtos.GrantResponse grant(RpcController controller,
-          AccessControlProtos.GrantRequest request) throws ServiceException {
-        return stub.grant(controller, request);
-      }
-
-      @Override
-      public AccessControlProtos.RevokeResponse revoke(RpcController controller,
-          AccessControlProtos.RevokeRequest request) throws ServiceException {
-        return stub.revoke(controller, request);
-      }
-
-      @Override
-      public GetUserPermissionsResponse getUserPermissions(RpcController controller,
-          GetUserPermissionsRequest request) throws ServiceException {
-        return stub.getUserPermissions(controller, request);
-      }
-
-      @Override
-      public HasUserPermissionsResponse hasUserPermissions(RpcController controller,
-          HasUserPermissionsRequest request) throws ServiceException {
-        return stub.hasUserPermissions(controller, request);
-      }
-    };
-  }
-
-  private static void release(MasterServiceState mss) {
-    if (mss != null && mss.connection != null) {
-      ((ConnectionImplementation)mss.connection).releaseMaster(mss);
-    }
-  }
-
-  private boolean isKeepAliveMasterConnectedAndRunning(MasterServiceState mss) {
-    if (mss.getStub() == null){
-      return false;
-    }
-    try {
-      return mss.isMasterRunning();
-    } catch (UndeclaredThrowableException e) {
-      // It's somehow messy, but we can receive exceptions such as
-      //  java.net.ConnectException but they're not declared. So we catch it...
-      LOG.info("Master connection is not running anymore", e.getUndeclaredThrowable());
-      return false;
-    } catch (IOException se) {
-      LOG.warn("Checking master connection", se);
-      return false;
-    }
-  }
-
-  void releaseMaster(MasterServiceState mss) {
-    if (mss.getStub() == null) {
-      return;
-    }
-    synchronized (masterLock) {
-      --mss.userCount;
-    }
-  }
-
-  private void closeMasterService(MasterServiceState mss) {
-    if (mss.getStub() != null) {
-      LOG.info("Closing master protocol: " + mss);
-      mss.clearStub();
-    }
-    mss.userCount = 0;
-  }
-
-  /**
-   * Immediate close of the shared master. Can be by the delayed close or when closing the
-   * connection itself.
-   */
-  private void closeMaster() {
-    synchronized (masterLock) {
-      closeMasterService(masterServiceState);
-    }
-  }
-
-  void updateCachedLocation(RegionInfo hri, ServerName source, ServerName serverName, long seqNum) {
-    HRegionLocation newHrl = new HRegionLocation(hri, serverName, seqNum);
-    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);
-  }
-
-  /**
-   * Update the location with the new value (if the exception is a RegionMovedException)
-   * or delete it from the cache. Does nothing if we can be sure from the exception that
-   * the location is still accurate, or if the cache has already been updated.
-   * @param exception an object (to simplify user code) on which we will try to find a nested
-   *   or wrapped or both RegionMovedException
-   * @param source server that is the source of the location update.
-   */
-  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) +
-          ", tableName=" + (tableName == null ? "null" : tableName));
-      return;
-    }
-
-    if (source == null) {
-      // This should not happen, but let's secure ourselves.
-      return;
-    }
-
-    if (regionName == null) {
-      // we do not know which region, so just remove the cache entry for the row and server
-      if (metrics != null) {
-        metrics.incrCacheDroppingExceptions(exception);
-      }
-      metaCache.clearCache(tableName, rowkey, source);
-      return;
-    }
-
-    // Is it something we have already updated?
-    final RegionLocations oldLocations = getCachedLocation(tableName, rowkey);
-    HRegionLocation oldLocation = null;
-    if (oldLocations != null) {
-      oldLocation = oldLocations.getRegionLocationByRegionName(regionName);
-    }
-    if (oldLocation == null || !source.equals(oldLocation.getServerName())) {
-      // There is no such location in the cache (it's been removed already) or
-      // the cache has already been refreshed with a different location.  => nothing to do
-      return;
-    }
-
-    RegionInfo regionInfo = oldLocation.getRegion();
-    Throwable cause = ClientExceptionsUtil.findException(exception);
-    if (cause != null) {
-      if (!ClientExceptionsUtil.isMetaClearingException(cause)) {
-        // We know that the region is still on this region server
-        return;
-      }
-
-      if (cause instanceof RegionMovedException) {
-        RegionMovedException rme = (RegionMovedException) cause;
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Region " + regionInfo.getRegionNameAsString() + " moved to " +
-              rme.getHostname() + ":" + rme.getPort() +
-              " according to " + source.getAddress());
-        }
-        // We know that the region is not anymore on this region server, but we know
-        //  the new location.
-        updateCachedLocation(
-            regionInfo, source, rme.getServerName(), rme.getLocationSeqNum());
-        return;
-      }
-    }
-
-    if (metrics != null) {
-      metrics.incrCacheDroppingExceptions(exception);
-    }
-
-    // If we're here, it means that can cannot be sure about the location, so we remove it from
-    // the cache. Do not send the source because source can be a new server in the same host:port
-    metaCache.clearCache(regionInfo);
-  }
-
-  /**
-   * @return Default AsyncProcess associated with this connection.
-   */
-  public AsyncProcess getAsyncProcess() {
-    return asyncProcess;
-  }
-
-  /**
-   * @return the current statistics tracker associated with this connection
-   */
-  public ServerStatisticTracker getStatisticsTracker() {
-    return this.stats;
-  }
-
-  /**
-   * @return the configured client backoff policy
-   */
-  public ClientBackoffPolicy getBackoffPolicy() {
-    return this.backoffPolicy;
-  }
-
-  /*
-   * Return the number of cached region for a table. It will only be called
-   * from a unit test.
-   */
-  @VisibleForTesting
-  int getNumberOfCachedRegionLocations(final TableName tableName) {
-    return metaCache.getNumberOfCachedRegionLocations(tableName);
-  }
-
-  @Override
-  public void abort(final String msg, Throwable t) {
-    if (t != null) {
-      LOG.error(HBaseMarkers.FATAL, msg, t);
-    } else {
-      LOG.error(HBaseMarkers.FATAL, msg);
-    }
-    this.aborted = true;
-    close();
-    this.closed = true;
-  }
-
-  @Override
-  public boolean isClosed() {
-    return this.closed;
-  }
-
-  @Override
-  public boolean isAborted(){
-    return this.aborted;
-  }
-
-  /**
-   * @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());
-  }
-
-  @Override
-  public void close() {
-    if (this.closed) {
-      return;
-    }
-    closeMaster();
-    shutdownPools();
-    if (this.metrics != null) {
-      this.metrics.shutdown();
-    }
-    this.closed = true;
-    registry.close();
-    this.stubs.clear();
-    if (clusterStatusListener != null) {
-      clusterStatusListener.close();
-    }
-    if (rpcClient != null) {
-      rpcClient.close();
-    }
-    if (authService != null) {
-      authService.shutdown();
-    }
-  }
-
-  /**
-   * Close the connection for good. On the off chance that someone is unable to close
-   * the connection, perhaps because it bailed out prematurely, the method
-   * below will ensure that this instance is cleaned up.
-   * Caveat: The JVM may take an unknown amount of time to call finalize on an
-   * unreachable object, so our hope is that every consumer cleans up after
-   * itself, like any good citizen.
-   */
-  @Override
-  protected void finalize() throws Throwable {
-    super.finalize();
-    close();
-  }
-
-  /**
-   * @return Nonce generator for this ClusterConnection; may be null if disabled in configuration.
-   */
-  public NonceGenerator getNonceGenerator() {
-    return nonceGenerator;
-  }
-
-  /**
-   * 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);
-    if (tableState == null) {
-      throw new TableNotFoundException(tableName);
-    }
-    return tableState;
-  }
-
-  /**
-   * 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());
-  }
-
-  /**
-   * @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();
-  }
-
-  /**
-   * @return a ConnectionConfiguration object holding parsed configuration values
-   */
-  public ConnectionConfiguration getConnectionConfiguration() {
-    return this.connectionConfig;
-  }
-
-  /**
-   * @return Connection's RpcRetryingCallerFactory instance
-   */
-  public RpcRetryingCallerFactory getRpcRetryingCallerFactory() {
-    return this.rpcCallerFactory;
-  }
-
-  /**
-   * @return Connection's RpcControllerFactory instance
-   */
-  public RpcControllerFactory getRpcControllerFactory() {
-    return this.rpcControllerFactory;
-  }
-
-  private static <T> T get(CompletableFuture<T> future) throws IOException {
-    try {
-      return future.get();
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      throw (IOException) new InterruptedIOException().initCause(e);
-    } catch (ExecutionException e) {
-      Throwable cause = e.getCause();
-      Throwables.propagateIfPossible(cause, IOException.class);
-      throw new IOException(cause);
-    }
-  }
-
-  @Override
-  public AsyncConnection toAsyncConnection() {
-    throw new UnsupportedOperationException();
-  }
-}
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 a6c47b5..2ac3cd2 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
@@ -52,7 +52,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.util.Threads;
@@ -62,7 +61,6 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
@@ -113,16 +111,6 @@ public final class ConnectionUtils {
   }
 
   /**
-   * @param conn The connection for which to replace the generator.
-   * @param cnm Replaces the nonce generator used, for testing.
-   * @return old nonce generator.
-   */
-  public static NonceGenerator injectNonceGeneratorForTesting(ConnectionImplementation conn,
-      NonceGenerator cnm) {
-    return ConnectionImplementation.injectNonceGeneratorForTesting(conn, cnm);
-  }
-
-  /**
    * Changes the configuration to set the number of retries needed when using Connection internally,
    * e.g. for updating catalog tables, etc. Call this method before we create any Connections.
    * @param c The Configuration instance to set the retries into.
@@ -143,31 +131,6 @@ public final class ConnectionUtils {
   }
 
   /**
-   * Setup the connection class, so that it will not depend on master being online. Used for testing
-   * @param conf configuration to set
-   */
-  @VisibleForTesting
-  public static void setupMasterlessConnection(Configuration conf) {
-    conf.set(ConnectionUtils.HBASE_CLIENT_CONNECTION_IMPL, MasterlessConnection.class.getName());
-  }
-
-  /**
-   * Some tests shut down the master. But table availability is a master RPC which is performed on
-   * region re-lookups.
-   */
-  static class MasterlessConnection extends ConnectionImplementation {
-    MasterlessConnection(Configuration conf, ExecutorService pool, User user) throws IOException {
-      super(conf, pool, user);
-    }
-
-    @Override
-    public boolean isTableDisabled(TableName tableName) throws IOException {
-      // treat all tables as enabled
-      return false;
-    }
-  }
-
-  /**
    * Return retires + 1. The returned value will be in range [1, Integer.MAX_VALUE].
    */
   static int retries2Attempts(int retries) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DelayingRunner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DelayingRunner.java
deleted file mode 100644
index 8ab5d85..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DelayingRunner.java
+++ /dev/null
@@ -1,116 +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 org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
-import java.util.List;
-import java.util.Map;
-
-/**
- * A wrapper for a runnable for a group of actions for a single regionserver.
- * <p>
- * This can be used to build up the actions that should be taken and then
- * </p>
- * <p>
- * This class exists to simulate using a ScheduledExecutorService with just a regular
- * ExecutorService and Runnables. It is used for legacy reasons in the the client; this could
- * only be removed if we change the expectations in HTable around the pool the client is able to
- * pass in and even if we deprecate the current APIs would require keeping this class around
- * for the interim to bridge between the legacy ExecutorServices and the scheduled pool.
- * </p>
- */
-@InterfaceAudience.Private
-public class DelayingRunner implements Runnable {
-  private static final Logger LOG = LoggerFactory.getLogger(DelayingRunner.class);
-
-  private final Object sleepLock = new Object();
-  private boolean triggerWake = false;
-  private long sleepTime;
-  private MultiAction actions = new MultiAction();
-  private Runnable runnable;
-
-  public DelayingRunner(long sleepTime, Map.Entry<byte[], List<Action>> e) {
-    this.sleepTime = sleepTime;
-    add(e);
-  }
-
-  public void setRunner(Runnable runner) {
-    this.runnable = runner;
-  }
-
-  @Override
-  public void run() {
-    if (!sleep()) {
-      LOG.warn(
-          "Interrupted while sleeping for expected sleep time " + sleepTime + " ms");
-    }
-    //TODO maybe we should consider switching to a listenableFuture for the actual callable and
-    // then handling the results/errors as callbacks. That way we can decrement outstanding tasks
-    // even if we get interrupted here, but for now, we still need to run so we decrement the
-    // outstanding tasks
-    this.runnable.run();
-  }
-
-  /**
-   * Sleep for an expected amount of time.
-   * <p>
-   * This is nearly a copy of what the Sleeper does, but with the ability to know if you
-   * got interrupted while sleeping.
-   * </p>
-   *
-   * @return <tt>true</tt> if the sleep completely entirely successfully,
-   * but otherwise <tt>false</tt> if the sleep was interrupted.
-   */
-  private boolean sleep() {
-    long now = EnvironmentEdgeManager.currentTime();
-    long startTime = now;
-    long waitTime = sleepTime;
-    while (waitTime > 0) {
-      long woke = -1;
-      try {
-        synchronized (sleepLock) {
-          if (triggerWake) break;
-          sleepLock.wait(waitTime);
-        }
-        woke = EnvironmentEdgeManager.currentTime();
-      } catch (InterruptedException iex) {
-        return false;
-      }
-      // Recalculate waitTime.
-      woke = (woke == -1) ? EnvironmentEdgeManager.currentTime() : woke;
-      waitTime = waitTime - (woke - startTime);
-    }
-    return true;
-  }
-
-  public void add(Map.Entry<byte[], List<Action>> e) {
-    actions.add(e.getKey(), e.getValue());
-  }
-
-  public MultiAction getActions() {
-    return actions;
-  }
-
-  public long getSleepTime() {
-    return sleepTime;
-  }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FastFailInterceptorContext.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FastFailInterceptorContext.java
deleted file mode 100644
index 6b0e790..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FastFailInterceptorContext.java
+++ /dev/null
@@ -1,135 +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 org.apache.commons.lang3.mutable.MutableBoolean;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
-
-@InterfaceAudience.Private
-class FastFailInterceptorContext extends RetryingCallerInterceptorContext {
-
-  // The variable that indicates whether we were able to connect with the server
-  // in the last run
-  private MutableBoolean couldNotCommunicateWithServer = new MutableBoolean(false);
-
-  // If set, we guarantee that no modifications went to server
-  private MutableBoolean guaranteedClientSideOnly = new MutableBoolean(false);
-
-  // The variable which indicates whether this was a retry or the first time
-  private boolean didTry = false;
-
-  // The failure info that is associated with the machine which we are trying to
-  // contact as part of this attempt.
-  private FailureInfo fInfo = null;
-
-  // Variable indicating that the thread that is currently executing the
-  // operation is in a mode where it would retry instead of failing fast, so
-  // that we can figure out whether making contact with the server is
-  // possible or not.
-  private boolean retryDespiteFastFailMode = false;
-
-  // The server that would be contacted to successfully complete this operation.
-  private ServerName server;
-
-  // The number of the retry we are currenty doing.
-  private int tries;
-
-  public MutableBoolean getCouldNotCommunicateWithServer() {
-    return couldNotCommunicateWithServer;
-  }
-
-  public MutableBoolean getGuaranteedClientSideOnly() {
-    return guaranteedClientSideOnly;
-  }
-
-  public FailureInfo getFailureInfo() {
-    return fInfo;
-  }
-
-  public ServerName getServer() {
-    return server;
-  }
-
-  public int getTries() {
-    return tries;
-  }
-
-  public boolean didTry() {
-    return didTry;
-  }
-
-  public boolean isRetryDespiteFastFailMode() {
-    return retryDespiteFastFailMode;
-  }
-
-  public void setCouldNotCommunicateWithServer(
-      MutableBoolean couldNotCommunicateWithServer) {
-    this.couldNotCommunicateWithServer = couldNotCommunicateWithServer;
-  }
-
-  public void setGuaranteedClientSideOnly(MutableBoolean guaranteedClientSideOnly) {
-    this.guaranteedClientSideOnly = guaranteedClientSideOnly;
-  }
-
-  public void setDidTry(boolean didTry) {
-    this.didTry = didTry;
-  }
-
-  public void setFailureInfo(FailureInfo fInfo) {
-    this.fInfo = fInfo;
-  }
-
-  public void setRetryDespiteFastFailMode(boolean retryDespiteFastFailMode) {
-    this.retryDespiteFastFailMode = retryDespiteFastFailMode;
-  }
-
-  public void setServer(ServerName server) {
-    this.server = server;
-  }
-
-  public void setTries(int tries) {
-    this.tries = tries;
-  }
-
-  @Override
-  public void clear() {
-    server = null;
-    fInfo = null;
-    didTry = false;
-    couldNotCommunicateWithServer.setValue(false);
-    guaranteedClientSideOnly.setValue(false);
-    retryDespiteFastFailMode = false;
-    tries = 0;
-  }
-
-  @Override
-  public FastFailInterceptorContext prepare(RetryingCallable<?> callable) {
-    return prepare(callable, 0);
-  }
-
-  @Override
-  public FastFailInterceptorContext prepare(RetryingCallable<?> callable, int tries) {
-    if (callable instanceof RegionServerCallable) {
-      RegionServerCallable<?, ?> retryingCallable = (RegionServerCallable<?, ?>) callable;
-      server = retryingCallable.getLocation().getServerName();
-    }
-    this.tries = tries;
-    return this;
-  }
-}
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
deleted file mode 100644
index d881fe0..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
+++ /dev/null
@@ -1,88 +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.hbase.TableName;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.util.Bytes;
-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;
-
-/**
- * A Callable for flushRegion() RPC.
- */
-@InterfaceAudience.Private
-public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionResponse> {
-  private static final Logger LOG = LoggerFactory.getLogger(FlushRegionCallable.class);
-  private final byte[] regionName;
-  private final boolean writeFlushWalMarker;
-  private boolean reload;
-
-  public FlushRegionCallable(ConnectionImplementation connection,
-      RpcControllerFactory rpcControllerFactory, TableName tableName, byte[] regionName,
-      byte[] regionStartKey, boolean writeFlushWalMarker) {
-    super(connection, rpcControllerFactory, tableName, regionStartKey);
-    this.regionName = regionName;
-    this.writeFlushWalMarker = writeFlushWalMarker;
-  }
-
-  public FlushRegionCallable(ConnectionImplementation connection,
-      RpcControllerFactory rpcControllerFactory, RegionInfo regionInfo,
-      boolean writeFlushWalMarker) {
-    this(connection, rpcControllerFactory, regionInfo.getTable(), regionInfo.getRegionName(),
-      regionInfo.getStartKey(), writeFlushWalMarker);
-  }
-
-  @Override
-  public void prepare(boolean reload) throws IOException {
-    super.prepare(reload);
-    this.reload = reload;
-  }
-
-  @Override
-  protected FlushRegionResponse call(HBaseRpcController controller) throws Exception {
-    // Check whether we should still do the flush to this region. If the regions are changed due
-    // to splits or merges, etc return success
-    if (!Bytes.equals(location.getRegionInfo().getRegionName(), regionName)) {
-      if (!reload) {
-        throw new IOException("Cached location seems to be different than requested region.");
-      }
-      LOG.info("Skipping flush region, because the located region "
-          + Bytes.toStringBinary(location.getRegionInfo().getRegionName()) + " is different than "
-          + " requested region " + Bytes.toStringBinary(regionName));
-      return FlushRegionResponse.newBuilder()
-          .setLastFlushTime(EnvironmentEdgeManager.currentTime())
-          .setFlushed(false)
-          .setWroteFlushWalMarker(false)
-          .build();
-    }
-
-    FlushRegionRequest request =
-        RequestConverter.buildFlushRegionRequest(regionName, writeFlushWalMarker);
-    return stub.flushRegion(controller, request);
-  }
-}
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
deleted file mode 100644
index 9c62678..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ /dev/null
@@ -1,3921 +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 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;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-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.CacheEvictionStats;
-import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
-import org.apache.hadoop.hbase.ClusterMetrics;
-import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterMetricsBuilder;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.MasterNotRunningException;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.NamespaceNotFoundException;
-import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.RegionMetrics;
-import org.apache.hadoop.hbase.RegionMetricsBuilder;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableExistsException;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotDisabledException;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.UnknownRegionException;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
-import org.apache.hadoop.hbase.client.replication.TableCFs;
-import org.apache.hadoop.hbase.client.security.SecurityCapability;
-import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.quotas.QuotaFilter;
-import org.apache.hadoop.hbase.quotas.QuotaRetriever;
-import org.apache.hadoop.hbase.quotas.QuotaSettings;
-import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
-import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
-import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-import org.apache.hadoop.hbase.replication.SyncReplicationState;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
-import org.apache.hadoop.hbase.security.access.Permission;
-import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
-import org.apache.hadoop.hbase.security.access.UserPermission;
-import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
-import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
-import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
-import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
-import org.apache.hadoop.hbase.util.Addressing;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
-import org.apache.hbase.thirdparty.com.google.protobuf.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.AccessControlProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasUserPermissionsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeRequest;
-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.AdminProtos.ClearCompactionQueuesRequest;
-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.CompactRegionRequest;
-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.FlushRegionRequest;
-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.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.UpdateConfigurationRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRpcThrottleEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
-
-/**
- * HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
- * this is an HBase-internal class as defined in
- * https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/InterfaceClassification.html
- * There are no guarantees for backwards source / binary compatibility and methods or class can
- * change or go away without deprecation.
- * Use {@link Connection#getAdmin()} to obtain an instance of {@link Admin} instead of constructing
- * an HBaseAdmin directly.
- *
- * <p>Connection should be an <i>unmanaged</i> connection obtained via
- * {@link ConnectionFactory#createConnection(Configuration)}
- *
- * @see ConnectionFactory
- * @see Connection
- * @see Admin
- */
-@InterfaceAudience.Private
-public class HBaseAdmin implements Admin {
-  private static final Logger LOG = LoggerFactory.getLogger(HBaseAdmin.class);
-
-  private ConnectionImplementation connection;
-
-  private final Configuration conf;
-  private final long pause;
-  private final int numRetries;
-  private final int syncWaitTimeout;
-  private boolean aborted;
-  private int operationTimeout;
-  private int rpcTimeout;
-  private int getProcedureTimeout;
-
-  private RpcRetryingCallerFactory rpcCallerFactory;
-  private RpcControllerFactory rpcControllerFactory;
-
-  private NonceGenerator ng;
-
-  @Override
-  public int getOperationTimeout() {
-    return operationTimeout;
-  }
-
-  @Override
-  public int getSyncWaitTimeout() {
-    return syncWaitTimeout;
-  }
-
-  HBaseAdmin(ConnectionImplementation connection) throws IOException {
-    this.conf = connection.getConfiguration();
-    this.connection = connection;
-
-    // TODO: receive ConnectionConfiguration here rather than re-parsing these configs every time.
-    this.pause = this.conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
-        HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
-    this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
-        HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
-    this.operationTimeout = this.conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
-        HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
-    this.rpcTimeout = this.conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
-        HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
-    this.syncWaitTimeout = this.conf.getInt(
-      "hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
-    this.getProcedureTimeout =
-        this.conf.getInt("hbase.client.procedure.future.get.timeout.msec", 10 * 60000); // 10min
-
-    this.rpcCallerFactory = connection.getRpcRetryingCallerFactory();
-    this.rpcControllerFactory = connection.getRpcControllerFactory();
-
-    this.ng = this.connection.getNonceGenerator();
-  }
-
-  @Override
-  public void abort(String why, Throwable e) {
-    // Currently does nothing but throw the passed message and exception
-    this.aborted = true;
-    throw new RuntimeException(why, e);
-  }
-
-  @Override
-  public boolean isAborted() {
-    return this.aborted;
-  }
-
-  @Override
-  public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
-  throws IOException {
-    return get(abortProcedureAsync(procId, mayInterruptIfRunning), this.syncWaitTimeout,
-      TimeUnit.MILLISECONDS);
-  }
-
-  @Override
-  public Future<Boolean> abortProcedureAsync(final long procId, final boolean mayInterruptIfRunning)
-      throws IOException {
-    Boolean abortProcResponse =
-        executeCallable(new MasterCallable<AbortProcedureResponse>(getConnection(),
-            getRpcControllerFactory()) {
-      @Override
-      protected AbortProcedureResponse rpcCall() throws Exception {
-        AbortProcedureRequest abortProcRequest =
-            AbortProcedureRequest.newBuilder().setProcId(procId).build();
-        return master.abortProcedure(getRpcController(), abortProcRequest);
-      }
-    }).getIsProcedureAborted();
-    return new AbortProcedureFuture(this, procId, abortProcResponse);
-  }
-
-  @Override
-  public List<TableDescriptor> listTableDescriptors() throws IOException {
-    return listTableDescriptors((Pattern)null, false);
-  }
-
-  @Override
-  public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
-      throws IOException {
-    return executeCallable(new MasterCallable<List<TableDescriptor>>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected List<TableDescriptor> rpcCall() throws Exception {
-        GetTableDescriptorsRequest req =
-            RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
-        return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(),
-            req));
-      }
-    });
-  }
-
-  @Override
-  public TableDescriptor getDescriptor(TableName tableName)
-      throws TableNotFoundException, IOException {
-    return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
-       operationTimeout, rpcTimeout);
-  }
-
-  @Override
-  public Future<Void> modifyTableAsync(TableDescriptor td) throws IOException {
-    ModifyTableResponse response = executeCallable(
-      new MasterCallable<ModifyTableResponse>(getConnection(), getRpcControllerFactory()) {
-        Long nonceGroup = ng.getNonceGroup();
-        Long nonce = ng.newNonce();
-        @Override
-        protected ModifyTableResponse rpcCall() throws Exception {
-          setPriority(td.getTableName());
-          ModifyTableRequest request = RequestConverter.buildModifyTableRequest(
-            td.getTableName(), td, nonceGroup, nonce);
-          return master.modifyTable(getRpcController(), request);
-        }
-      });
-    return new ModifyTableFuture(this, td.getTableName(), response);
-  }
-
-  @Override
-  public List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException {
-    return executeCallable(new MasterCallable<List<TableDescriptor>>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected List<TableDescriptor> rpcCall() throws Exception {
-        return master.listTableDescriptorsByNamespace(getRpcController(),
-                ListTableDescriptorsByNamespaceRequest.newBuilder()
-                  .setNamespaceName(Bytes.toString(name)).build())
-                .getTableSchemaList()
-                .stream()
-                .map(ProtobufUtil::toTableDescriptor)
-                .collect(Collectors.toList());
-      }
-    });
-  }
-
-  @Override
-  public List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException {
-    return executeCallable(new MasterCallable<List<TableDescriptor>>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected List<TableDescriptor> rpcCall() throws Exception {
-        GetTableDescriptorsRequest req =
-            RequestConverter.buildGetTableDescriptorsRequest(tableNames);
-          return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(),
-              req));
-      }
-    });
-  }
-
-  @Override
-  public List<RegionInfo> getRegions(final ServerName sn) throws IOException {
-    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    // TODO: There is no timeout on this controller. Set one!
-    HBaseRpcController controller = rpcControllerFactory.newController();
-    return ProtobufUtil.getOnlineRegions(controller, admin);
-  }
-
-  @Override
-  public List<RegionInfo> getRegions(TableName tableName) throws IOException {
-    if (TableName.isMetaTableName(tableName)) {
-      return Arrays.asList(RegionInfoBuilder.FIRST_META_REGIONINFO);
-    } else {
-      return MetaTableAccessor.getTableRegions(connection, tableName, true);
-    }
-  }
-
-  private static class AbortProcedureFuture extends ProcedureFuture<Boolean> {
-    private boolean isAbortInProgress;
-
-    public AbortProcedureFuture(
-        final HBaseAdmin admin,
-        final Long procId,
-        final Boolean abortProcResponse) {
-      super(admin, procId);
-      this.isAbortInProgress = abortProcResponse;
-    }
-
-    @Override
-    public Boolean get(long timeout, TimeUnit unit)
-        throws InterruptedException, ExecutionException, TimeoutException {
-      if (!this.isAbortInProgress) {
-        return false;
-      }
-      super.get(timeout, unit);
-      return true;
-    }
-  }
-
-  /** @return Connection used by this object. */
-  @Override
-  public ConnectionImplementation getConnection() {
-    return connection;
-  }
-
-  @Override
-  public boolean tableExists(final TableName tableName) throws IOException {
-    return executeCallable(new RpcRetryingCallable<Boolean>() {
-      @Override
-      protected Boolean rpcCall(int callTimeout) throws Exception {
-        return MetaTableAccessor.tableExists(connection, tableName);
-      }
-    });
-  }
-
-  @Override
-  public TableName[] listTableNames() throws IOException {
-    return listTableNames((Pattern)null, false);
-  }
-
-  @Override
-  public TableName[] listTableNames(Pattern pattern) throws IOException {
-    return listTableNames(pattern, false);
-  }
-
-  @Override
-  public TableName[] listTableNames(final Pattern pattern, final boolean includeSysTables)
-      throws IOException {
-    return executeCallable(new MasterCallable<TableName[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected TableName[] rpcCall() throws Exception {
-        GetTableNamesRequest req =
-            RequestConverter.buildGetTableNamesRequest(pattern, includeSysTables);
-        return ProtobufUtil.getTableNameArray(master.getTableNames(getRpcController(), req)
-            .getTableNamesList());
-      }
-    });
-  }
-
-  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 =
-            RequestConverter.buildGetTableDescriptorsRequest(tableName);
-          GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
-          if (!htds.getTableSchemaList().isEmpty()) {
-            return ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0));
-          }
-          return null;
-        }
-      }, rpcCallerFactory, operationTimeout, rpcTimeout);
-    if (td != null) {
-      return td;
-    }
-    throw new TableNotFoundException(tableName.getNameAsString());
-  }
-
-  private long getPauseTime(int tries) {
-    int triesCount = tries;
-    if (triesCount >= HConstants.RETRY_BACKOFF.length) {
-      triesCount = HConstants.RETRY_BACKOFF.length - 1;
-    }
-    return this.pause * HConstants.RETRY_BACKOFF[triesCount];
-  }
-
-  @Override
-  public void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
-      throws IOException {
-    if (numRegions < 3) {
-      throw new IllegalArgumentException("Must create at least three regions");
-    } else if (Bytes.compareTo(startKey, endKey) >= 0) {
-      throw new IllegalArgumentException("Start key must be smaller than end key");
-    }
-    if (numRegions == 3) {
-      createTable(desc, new byte[][] { startKey, endKey });
-      return;
-    }
-    byte[][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
-    if (splitKeys == null || splitKeys.length != numRegions - 1) {
-      throw new IllegalArgumentException("Unable to split key range into enough regions");
-    }
-    createTable(desc, splitKeys);
-  }
-
-  @Override
-  public Future<Void> createTableAsync(final TableDescriptor desc, final byte[][] splitKeys)
-      throws IOException {
-    if (desc.getTableName() == null) {
-      throw new IllegalArgumentException("TableName cannot be null");
-    }
-    if (splitKeys != null && splitKeys.length > 0) {
-      Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR);
-      // Verify there are no duplicate split keys
-      byte[] lastKey = null;
-      for (byte[] splitKey : splitKeys) {
-        if (Bytes.compareTo(splitKey, HConstants.EMPTY_BYTE_ARRAY) == 0) {
-          throw new IllegalArgumentException(
-              "Empty split key must not be passed in the split keys.");
-        }
-        if (lastKey != null && Bytes.equals(splitKey, lastKey)) {
-          throw new IllegalArgumentException("All split keys must be unique, " +
-            "found duplicate: " + Bytes.toStringBinary(splitKey) +
-            ", " + Bytes.toStringBinary(lastKey));
-        }
-        lastKey = splitKey;
-      }
-    }
-
-    CreateTableResponse response = executeCallable(
-      new MasterCallable<CreateTableResponse>(getConnection(), getRpcControllerFactory()) {
-        Long nonceGroup = ng.getNonceGroup();
-        Long nonce = ng.newNonce();
-        @Override
-        protected CreateTableResponse rpcCall() throws Exception {
-          setPriority(desc.getTableName());
-          CreateTableRequest request = RequestConverter.buildCreateTableRequest(
-            desc, splitKeys, nonceGroup, nonce);
-          return master.createTable(getRpcController(), request);
-        }
-      });
-    return new CreateTableFuture(this, desc, splitKeys, response);
-  }
-
-  private static class CreateTableFuture extends TableFuture<Void> {
-    private final TableDescriptor desc;
-    private final byte[][] splitKeys;
-
-    public CreateTableFuture(final HBaseAdmin admin, final TableDescriptor desc,
-        final byte[][] splitKeys, final CreateTableResponse response) {
-      super(admin, desc.getTableName(),
-              (response != null && response.hasProcId()) ? response.getProcId() : null);
-      this.splitKeys = splitKeys;
-      this.desc = desc;
-    }
-
-    @Override
-    protected TableDescriptor getDescriptor() {
-      return desc;
-    }
-
-    @Override
-    public String getOperationType() {
-      return "CREATE";
-    }
-
-    @Override
-    protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException {
-      waitForTableEnabled(deadlineTs);
-      waitForAllRegionsOnline(deadlineTs, splitKeys);
-      return null;
-    }
-  }
-
-  @Override
-  public Future<Void> deleteTableAsync(final TableName tableName) throws IOException {
-    DeleteTableResponse response = executeCallable(
-      new MasterCallable<DeleteTableResponse>(getConnection(), getRpcControllerFactory()) {
-        Long nonceGroup = ng.getNonceGroup();
-        Long nonce = ng.newNonce();
-        @Override
-        protected DeleteTableResponse rpcCall() throws Exception {
-          setPriority(tableName);
-          DeleteTableRequest req =
-              RequestConverter.buildDeleteTableRequest(tableName, nonceGroup,nonce);
-          return master.deleteTable(getRpcController(), req);
-        }
-      });
-    return new DeleteTableFuture(this, tableName, response);
-  }
-
-  private static class DeleteTableFuture extends TableFuture<Void> {
-    public DeleteTableFuture(final HBaseAdmin admin, final TableName tableName,
-        final DeleteTableResponse response) {
-      super(admin, tableName,
-              (response != null && response.hasProcId()) ? response.getProcId() : null);
-    }
-
-    @Override
-    public String getOperationType() {
-      return "DELETE";
-    }
-
-    @Override
-    protected Void waitOperationResult(final long deadlineTs)
-        throws IOException, TimeoutException {
-      waitTableNotFound(deadlineTs);
-      return null;
-    }
-
-    @Override
-    protected Void postOperationResult(final Void result, final long deadlineTs)
-        throws IOException, TimeoutException {
-      // Delete cached information to prevent clients from using old locations
-      try (RegionLocator locator = getAdmin().getConnection().getRegionLocator(getTableName())) {
-        locator.clearRegionLocationCache();
-      }
-      return super.postOperationResult(result, deadlineTs);
-    }
-  }
-
-  @Override
-  public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
-      throws IOException {
-    TruncateTableResponse response =
-        executeCallable(new MasterCallable<TruncateTableResponse>(getConnection(),
-            getRpcControllerFactory()) {
-          Long nonceGroup = ng.getNonceGroup();
-          Long nonce = ng.newNonce();
-          @Override
-          protected TruncateTableResponse rpcCall() throws Exception {
-            setPriority(tableName);
-            LOG.info("Started truncating " + tableName);
-            TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
-              tableName, preserveSplits, nonceGroup, nonce);
-            return master.truncateTable(getRpcController(), req);
-          }
-        });
-    return new TruncateTableFuture(this, tableName, preserveSplits, response);
-  }
-
-  private static class TruncateTableFuture extends TableFuture<Void> {
-    private final boolean preserveSplits;
-
-    public TruncateTableFuture(final HBaseAdmin admin, final TableName tableName,
-        final boolean preserveSplits, final TruncateTableResponse response) {
-      super(admin, tableName,
-             (response != null && response.hasProcId()) ? response.getProcId() : null);
-      this.preserveSplits = preserveSplits;
-    }
-
-    @Override
-    public String getOperationType() {
-      return "TRUNCATE";
-    }
-
-    @Override
-    protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException {
-      waitForTableEnabled(deadlineTs);
-      // once the table is enabled, we know the operation is done. so we can fetch the splitKeys
-      byte[][] splitKeys = preserveSplits ? getAdmin().getTableSplits(getTableName()) : null;
-      waitForAllRegionsOnline(deadlineTs, splitKeys);
-      return null;
-    }
-  }
-
-  private byte[][] getTableSplits(final TableName tableName) throws IOException {
-    byte[][] splits = null;
-    try (RegionLocator locator = getConnection().getRegionLocator(tableName)) {
-      byte[][] startKeys = locator.getStartKeys();
-      if (startKeys.length == 1) {
-        return splits;
-      }
-      splits = new byte[startKeys.length - 1][];
-      for (int i = 1; i < startKeys.length; i++) {
-        splits[i - 1] = startKeys[i];
-      }
-    }
-    return splits;
-  }
-
-  @Override
-  public Future<Void> enableTableAsync(final TableName tableName) throws IOException {
-    TableName.isLegalFullyQualifiedTableName(tableName.getName());
-    EnableTableResponse response = executeCallable(
-      new MasterCallable<EnableTableResponse>(getConnection(), getRpcControllerFactory()) {
-        Long nonceGroup = ng.getNonceGroup();
-        Long nonce = ng.newNonce();
-        @Override
-        protected EnableTableResponse rpcCall() throws Exception {
-          setPriority(tableName);
-          LOG.info("Started enable of " + tableName);
-          EnableTableRequest req =
-              RequestConverter.buildEnableTableRequest(tableName, nonceGroup, nonce);
-          return master.enableTable(getRpcController(),req);
-        }
-      });
-    return new EnableTableFuture(this, tableName, response);
-  }
-
-  private static class EnableTableFuture extends TableFuture<Void> {
-    public EnableTableFuture(final HBaseAdmin admin, final TableName tableName,
-        final EnableTableResponse response) {
-      super(admin, tableName,
-              (response != null && response.hasProcId()) ? response.getProcId() : null);
-    }
-
-    @Override
-    public String getOperationType() {
-      return "ENABLE";
-    }
-
-    @Override
-    protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException {
-      waitForTableEnabled(deadlineTs);
-      return null;
-    }
-  }
-
-  @Override
-  public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
-    TableName.isLegalFullyQualifiedTableName(tableName.getName());
-    DisableTableResponse response = executeCallable(
-      new MasterCallable<DisableTableResponse>(getConnection(), getRpcControllerFactory()) {
-        Long nonceGroup = ng.getNonceGroup();
-        Long nonce = ng.newNonce();
-        @Override
-        protected DisableTableResponse rpcCall() throws Exception {
-          setPriority(tableName);
-          LOG.info("Started disable of " + tableName);
-          DisableTableRequest req =
-              RequestConverter.buildDisableTableRequest(
-                tableName, nonceGroup, nonce);
-          return master.disableTable(getRpcController(), req);
-        }
-      });
-    return new DisableTableFuture(this, tableName, response);
-  }
-
-  private static class DisableTableFuture extends TableFuture<Void> {
-    public DisableTableFuture(final HBaseAdmin admin, final TableName tableName,
-        final DisableTableResponse response) {
-      super(admin, tableName,
-              (response != null && response.hasProcId()) ? response.getProcId() : null);
-    }
-
-    @Override
-    public String getOperationType() {
-      return "DISABLE";
-    }
-
-    @Override
-    protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutException {
-      waitForTableDisabled(deadlineTs);
-      return null;
-    }
-  }
-
-  @Override
-  public boolean isTableEnabled(final TableName tableName) throws IOException {
-    checkTableExists(tableName);
-    return executeCallable(new RpcRetryingCallable<Boolean>() {
-      @Override
-      protected Boolean rpcCall(int callTimeout) throws Exception {
-        TableState tableState = MetaTableAccessor.getTableState(getConnection(), tableName);
-        if (tableState == null) {
-          throw new TableNotFoundException(tableName);
-        }
-        return tableState.inStates(TableState.State.ENABLED);
-      }
-    });
-  }
-
-  @Override
-  public boolean isTableDisabled(TableName tableName) throws IOException {
-    checkTableExists(tableName);
-    return connection.isTableDisabled(tableName);
-  }
-
-  @Override
-  public boolean isTableAvailable(TableName tableName) throws IOException {
-    return connection.isTableAvailable(tableName, null);
-  }
-
-  @Override
-  public Future<Void> addColumnFamilyAsync(final TableName tableName,
-      final ColumnFamilyDescriptor columnFamily) throws IOException {
-    AddColumnResponse response =
-        executeCallable(new MasterCallable<AddColumnResponse>(getConnection(),
-            getRpcControllerFactory()) {
-          Long nonceGroup = ng.getNonceGroup();
-          Long nonce = ng.newNonce();
-          @Override
-          protected AddColumnResponse rpcCall() throws Exception {
-            setPriority(tableName);
-            AddColumnRequest req =
-                RequestConverter.buildAddColumnRequest(tableName, columnFamily, nonceGroup, nonce);
-            return master.addColumn(getRpcController(), req);
-          }
-        });
-    return new AddColumnFamilyFuture(this, tableName, response);
-  }
-
-  private static class AddColumnFamilyFuture extends ModifyTableFuture {
-    public AddColumnFamilyFuture(final HBaseAdmin admin, final TableName tableName,
-        final AddColumnResponse response) {
-      super(admin, tableName, (response != null && response.hasProcId()) ? response.getProcId()
-          : null);
-    }
-
-    @Override
-    public String getOperationType() {
-      return "ADD_COLUMN_FAMILY";
-    }
-  }
-
-  @Override
-  public Future<Void> deleteColumnFamilyAsync(final TableName tableName, final byte[] columnFamily)
-      throws IOException {
-    DeleteColumnResponse response =
-        executeCallable(new MasterCallable<DeleteColumnResponse>(getConnection(),
-            getRpcControllerFactory()) {
-          Long nonceGroup = ng.getNonceGroup();
-          Long nonce = ng.newNonce();
-          @Override
-          protected DeleteColumnResponse rpcCall() throws Exception {
-            setPriority(tableName);
-            DeleteColumnRequest req =
-                RequestConverter.buildDeleteColumnRequest(tableName, columnFamily,
-                  nonceGroup, nonce);
-            return master.deleteColumn(getRpcController(), req);
-          }
-        });
-    return new DeleteColumnFamilyFuture(this, tableName, response);
-  }
-
-  private static class DeleteColumnFamilyFuture extends ModifyTableFuture {
-    public DeleteColumnFamilyFuture(final HBaseAdmin admin, final TableName tableName,
-        final DeleteColumnResponse response) {
-      super(admin, tableName, (response != null && response.hasProcId()) ? response.getProcId()
-          : null);
-    }
-
-    @Override
-    public String getOperationType() {
-      return "DELETE_COLUMN_FAMILY";
-    }
-  }
-
-  @Override
-  public Future<Void> modifyColumnFamilyAsync(final TableName tableName,
-      final ColumnFamilyDescriptor columnFamily) throws IOException {
-    ModifyColumnResponse response =
-        executeCallable(new MasterCallable<ModifyColumnResponse>(getConnection(),
-            getRpcControllerFactory()) {
-          Long nonceGroup = ng.getNonceGroup();
-          Long nonce = ng.newNonce();
-          @Override
-          protected ModifyColumnResponse rpcCall() throws Exception {
-            setPriority(tableName);
-            ModifyColumnRequest req =
-                RequestConverter.buildModifyColumnRequest(tableName, columnFamily,
-                  nonceGroup, nonce);
-            return master.modifyColumn(getRpcController(), req);
-          }
-        });
-    return new ModifyColumnFamilyFuture(this, tableName, response);
-  }
-
-  private static class ModifyColumnFamilyFuture extends ModifyTableFuture {
-    public ModifyColumnFamilyFuture(final HBaseAdmin admin, final TableName tableName,
-        final ModifyColumnResponse response) {
-      super(admin, tableName, (response != null && response.hasProcId()) ? response.getProcId()
-          : null);
-    }
-
-    @Override
-    public String getOperationType() {
-      return "MODIFY_COLUMN_FAMILY";
-    }
-  }
-
-  @Override
-  public void flush(final TableName tableName) throws IOException {
-    checkTableExists(tableName);
-    if (isTableDisabled(tableName)) {
-      LOG.info("Table is disabled: " + tableName.getNameAsString());
-      return;
-    }
-    execProcedure("flush-table-proc", tableName.getNameAsString(), new HashMap<>());
-  }
-
-  @Override
-  public void flushRegion(final byte[] regionName) throws IOException {
-    Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
-    if (regionServerPair == null) {
-      throw new IllegalArgumentException("Unknown regionname: " + Bytes.toStringBinary(regionName));
-    }
-    if (regionServerPair.getSecond() == null) {
-      throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
-    }
-    final RegionInfo regionInfo = regionServerPair.getFirst();
-    ServerName serverName = regionServerPair.getSecond();
-    flush(this.connection.getAdmin(serverName), regionInfo);
-  }
-
-  private void flush(AdminService.BlockingInterface admin, final RegionInfo info)
-    throws IOException {
-    ProtobufUtil.call(() -> {
-      // TODO: There is no timeout on this controller. Set one!
-      HBaseRpcController controller = rpcControllerFactory.newController();
-      FlushRegionRequest request =
-        RequestConverter.buildFlushRegionRequest(info.getRegionName());
-      admin.flushRegion(controller, request);
-      return null;
-    });
-  }
-
-  @Override
-  public void flushRegionServer(ServerName serverName) throws IOException {
-    for (RegionInfo region : getRegions(serverName)) {
-      flush(this.connection.getAdmin(serverName), region);
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void compact(final TableName tableName)
-    throws IOException {
-    compact(tableName, null, false, CompactType.NORMAL);
-  }
-
-  @Override
-  public void compactRegion(final byte[] regionName)
-    throws IOException {
-    compactRegion(regionName, null, false);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void compact(final TableName tableName, final byte[] columnFamily)
-    throws IOException {
-    compact(tableName, columnFamily, false, CompactType.NORMAL);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void compactRegion(final byte[] regionName, final byte[] columnFamily)
-    throws IOException {
-    compactRegion(regionName, columnFamily, false);
-  }
-
-  @Override
-  public Map<ServerName, Boolean> compactionSwitch(boolean switchState, List<String>
-      serverNamesList) throws IOException {
-    List<ServerName> serverList = new ArrayList<>();
-    if (serverNamesList.isEmpty()) {
-      ClusterMetrics status = getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS));
-      serverList.addAll(status.getLiveServerMetrics().keySet());
-    } else {
-      for (String regionServerName: serverNamesList) {
-        ServerName serverName = null;
-        try {
-          serverName = ServerName.valueOf(regionServerName);
-        } catch (Exception e) {
-          throw new IllegalArgumentException(String.format("Invalid ServerName format: %s",
-              regionServerName));
-        }
-        if (serverName == null) {
-          throw new IllegalArgumentException(String.format("Null ServerName: %s",
-              regionServerName));
-        }
-        serverList.add(serverName);
-      }
-    }
-    Map<ServerName, Boolean> res = new HashMap<>(serverList.size());
-    for (ServerName serverName: serverList) {
-      boolean prev_state = switchCompact(this.connection.getAdmin(serverName), switchState);
-      res.put(serverName, prev_state);
-    }
-    return res;
-  }
-
-  private Boolean switchCompact(AdminService.BlockingInterface admin, boolean onOrOff)
-      throws IOException {
-    return executeCallable(new RpcRetryingCallable<Boolean>() {
-      @Override protected Boolean rpcCall(int callTimeout) throws Exception {
-        HBaseRpcController controller = rpcControllerFactory.newController();
-        CompactionSwitchRequest request =
-            CompactionSwitchRequest.newBuilder().setEnabled(onOrOff).build();
-        CompactionSwitchResponse compactionSwitchResponse =
-            admin.compactionSwitch(controller, request);
-        return compactionSwitchResponse.getPrevState();
-      }
-    });
-  }
-
-  @Override
-  public void compactRegionServer(final ServerName serverName) throws IOException {
-    for (RegionInfo region : getRegions(serverName)) {
-      compact(this.connection.getAdmin(serverName), region, false, null);
-    }
-  }
-
-  @Override
-  public void majorCompactRegionServer(final ServerName serverName) throws IOException {
-    for (RegionInfo region : getRegions(serverName)) {
-      compact(this.connection.getAdmin(serverName), region, true, null);
-    }
-  }
-
-  @Override
-  public void majorCompact(final TableName tableName)
-  throws IOException {
-    compact(tableName, null, true, CompactType.NORMAL);
-  }
-
-  @Override
-  public void majorCompactRegion(final byte[] regionName)
-  throws IOException {
-    compactRegion(regionName, null, true);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void majorCompact(final TableName tableName, final byte[] columnFamily)
-  throws IOException {
-    compact(tableName, columnFamily, true, CompactType.NORMAL);
-  }
-
-  @Override
-  public void majorCompactRegion(final byte[] regionName, final byte[] columnFamily)
-  throws IOException {
-    compactRegion(regionName, columnFamily, true);
-  }
-
-  /**
-   * Compact a table.
-   * Asynchronous operation.
-   *
-   * @param tableName table or region to compact
-   * @param columnFamily column family within a table or region
-   * @param major True if we are to do a major compaction.
-   * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
-   * @throws IOException if a remote or network exception occurs
-   */
-  private void compact(final TableName tableName, final byte[] columnFamily,final boolean major,
-                       CompactType compactType) throws IOException {
-    switch (compactType) {
-      case MOB:
-        compact(this.connection.getAdminForMaster(), RegionInfo.createMobRegionInfo(tableName),
-            major, columnFamily);
-        break;
-      case NORMAL:
-        checkTableExists(tableName);
-        for (HRegionLocation loc :connection.locateRegions(tableName, false, false)) {
-          ServerName sn = loc.getServerName();
-          if (sn == null) {
-            continue;
-          }
-          try {
-            compact(this.connection.getAdmin(sn), loc.getRegion(), major, columnFamily);
-          } catch (NotServingRegionException e) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Trying to" + (major ? " major" : "") + " compact " + loc.getRegion() +
-                  ": " + StringUtils.stringifyException(e));
-            }
-          }
-        }
-        break;
-      default:
-        throw new IllegalArgumentException("Unknown compactType: " + compactType);
-    }
-  }
-
-  /**
-   * Compact an individual region.
-   * Asynchronous operation.
-   *
-   * @param regionName region to compact
-   * @param columnFamily column family within a table or region
-   * @param major True if we are to do a major compaction.
-   * @throws IOException if a remote or network exception occurs
-   * @throws InterruptedException
-   */
-  private void compactRegion(final byte[] regionName, final byte[] columnFamily,
-      final boolean major) throws IOException {
-    Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
-    if (regionServerPair == null) {
-      throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
-    }
-    if (regionServerPair.getSecond() == null) {
-      throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
-    }
-    compact(this.connection.getAdmin(regionServerPair.getSecond()), regionServerPair.getFirst(),
-      major, columnFamily);
-  }
-
-  private void compact(AdminService.BlockingInterface admin, RegionInfo hri, boolean major,
-      byte[] family) throws IOException {
-    Callable<Void> callable = new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        // TODO: There is no timeout on this controller. Set one!
-        HBaseRpcController controller = rpcControllerFactory.newController();
-        CompactRegionRequest request =
-            RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
-        admin.compactRegion(controller, request);
-        return null;
-      }
-    };
-    ProtobufUtil.call(callable);
-  }
-
-  @Override
-  public void move(byte[] encodedRegionName) throws IOException {
-    move(encodedRegionName, null);
-  }
-
-  public void move(byte[] encodedRegionName, ServerName destServerName) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        setPriority(encodedRegionName);
-        MoveRegionRequest request =
-          RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
-        master.moveRegion(getRpcController(), request);
-        return null;
-      }
-    });
-  }
-
-  @Override
-  public void assign(final byte [] regionName) throws MasterNotRunningException,
-      ZooKeeperConnectionException, IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        setPriority(regionName);
-        AssignRegionRequest request =
-            RequestConverter.buildAssignRegionRequest(getRegionName(regionName));
-        master.assignRegion(getRpcController(), request);
-        return null;
-      }
-    });
-  }
-
-  @Override
-  public void unassign(final byte [] regionName, final boolean force) throws IOException {
-    final byte[] toBeUnassigned = getRegionName(regionName);
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        setPriority(regionName);
-        UnassignRegionRequest request =
-            RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
-        master.unassignRegion(getRpcController(), request);
-        return null;
-      }
-    });
-  }
-
-  @Override
-  public void offline(final byte [] regionName)
-  throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        setPriority(regionName);
-        master.offlineRegion(getRpcController(),
-            RequestConverter.buildOfflineRegionRequest(regionName));
-        return null;
-      }
-    });
-  }
-
-  @Override
-  public boolean balancerSwitch(final boolean on, final boolean synchronous)
-  throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Boolean rpcCall() throws Exception {
-        SetBalancerRunningRequest req =
-            RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
-        return master.setBalancerRunning(getRpcController(), req).getPrevBalanceValue();
-      }
-    });
-  }
-
-  @Override
-  public boolean balance() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Boolean rpcCall() throws Exception {
-        return master.balance(getRpcController(),
-            RequestConverter.buildBalanceRequest(false)).getBalancerRan();
-      }
-    });
-  }
-
-  @Override
-  public boolean balance(final boolean force) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Boolean rpcCall() throws Exception {
-        return master.balance(getRpcController(),
-            RequestConverter.buildBalanceRequest(force)).getBalancerRan();
-      }
-    });
-  }
-
-  @Override
-  public boolean isBalancerEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Boolean rpcCall() throws Exception {
-        return master.isBalancerEnabled(getRpcController(),
-          RequestConverter.buildIsBalancerEnabledRequest()).getEnabled();
-      }
-    });
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public CacheEvictionStats clearBlockCache(final TableName tableName) throws IOException {
-    checkTableExists(tableName);
-    CacheEvictionStatsBuilder cacheEvictionStats = CacheEvictionStats.builder();
-    List<Pair<RegionInfo, ServerName>> pairs =
-      MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
-    Map<ServerName, List<RegionInfo>> regionInfoByServerName =
-        pairs.stream()
-            .filter(pair -> !(pair.getFirst().isOffline()))
-            .filter(pair -> pair.getSecond() != null)
-            .collect(Collectors.groupingBy(pair -> pair.getSecond(),
-                Collectors.mapping(pair -> pair.getFirst(), Collectors.toList())));
-
-    for (Map.Entry<ServerName, List<RegionInfo>> entry : regionInfoByServerName.entrySet()) {
-      CacheEvictionStats stats = clearBlockCache(entry.getKey(), entry.getValue());
-      cacheEvictionStats = cacheEvictionStats.append(stats);
-      if (stats.getExceptionCount() > 0) {
-        for (Map.Entry<byte[], Throwable> exception : stats.getExceptions().entrySet()) {
-          LOG.debug("Failed to clear block cache for "
-              + Bytes.toStringBinary(exception.getKey())
-              + " on " + entry.getKey() + ": ", exception.getValue());
-        }
-      }
-    }
-    return cacheEvictionStats.build();
-  }
-
-  private CacheEvictionStats clearBlockCache(final ServerName sn, final List<RegionInfo> hris)
-      throws IOException {
-    HBaseRpcController controller = rpcControllerFactory.newController();
-    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    ClearRegionBlockCacheRequest request =
-      RequestConverter.buildClearRegionBlockCacheRequest(hris);
-    ClearRegionBlockCacheResponse response;
-    try {
-      response = admin.clearRegionBlockCache(controller, request);
-      return ProtobufUtil.toCacheEvictionStats(response.getStats());
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
-  }
-
-  /**
-   * Invoke region normalizer. Can NOT run for various reasons.  Check logs.
-   *
-   * @return True if region normalizer ran, false otherwise.
-   */
-  @Override
-  public boolean normalize() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Boolean rpcCall() throws Exception {
-        return master.normalize(getRpcController(),
-            RequestConverter.buildNormalizeRequest()).getNormalizerRan();
-      }
-    });
-  }
-
-  @Override
-  public boolean isNormalizerEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Boolean rpcCall() throws Exception {
-        return master.isNormalizerEnabled(getRpcController(),
-          RequestConverter.buildIsNormalizerEnabledRequest()).getEnabled();
-      }
-    });
-  }
-
-  @Override
-  public boolean normalizerSwitch(final boolean on) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Boolean rpcCall() throws Exception {
-        SetNormalizerRunningRequest req =
-          RequestConverter.buildSetNormalizerRunningRequest(on);
-        return master.setNormalizerRunning(getRpcController(), req).getPrevNormalizerValue();
-      }
-    });
-  }
-
-  @Override
-  public boolean catalogJanitorSwitch(final boolean enable) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Boolean rpcCall() throws Exception {
-        return master.enableCatalogJanitor(getRpcController(),
-          RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
-      }
-    });
-  }
-
-  @Override
-  public int runCatalogJanitor() throws IOException {
-    return executeCallable(new MasterCallable<Integer>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Integer rpcCall() throws Exception {
-        return master.runCatalogScan(getRpcController(),
-          RequestConverter.buildCatalogScanRequest()).getScanResult();
-      }
-    });
-  }
-
-  @Override
-  public boolean isCatalogJanitorEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Boolean rpcCall() throws Exception {
-        return master.isCatalogJanitorEnabled(getRpcController(),
-          RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
-      }
-    });
-  }
-
-  @Override
-  public boolean cleanerChoreSwitch(final boolean on) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override public Boolean rpcCall() throws Exception {
-        return master.setCleanerChoreRunning(getRpcController(),
-            RequestConverter.buildSetCleanerChoreRunningRequest(on)).getPrevValue();
-      }
-    });
-  }
-
-  @Override
-  public boolean runCleanerChore() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override public Boolean rpcCall() throws Exception {
-        return master.runCleanerChore(getRpcController(),
-            RequestConverter.buildRunCleanerChoreRequest()).getCleanerChoreRan();
-      }
-    });
-  }
-
-  @Override
-  public boolean isCleanerChoreEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override public Boolean rpcCall() throws Exception {
-        return master.isCleanerChoreEnabled(getRpcController(),
-            RequestConverter.buildIsCleanerChoreEnabledRequest()).getValue();
-      }
-    });
-  }
-
-  /**
-   * Merge two regions. Synchronous operation.
-   * Note: It is not feasible to predict the length of merge.
-   *   Therefore, this is for internal testing only.
-   * @param nameOfRegionA encoded or full name of region a
-   * @param nameOfRegionB encoded or full name of region b
-   * @param forcible true if do a compulsory merge, otherwise we will only merge
-   *          two adjacent regions
-   * @throws IOException if a remote or network exception occurs
-   */
-  @VisibleForTesting
-  public void mergeRegionsSync(
-      final byte[] nameOfRegionA,
-      final byte[] nameOfRegionB,
-      final boolean forcible) throws IOException {
-    get(
-      mergeRegionsAsync(nameOfRegionA, nameOfRegionB, forcible),
-      syncWaitTimeout,
-      TimeUnit.MILLISECONDS);
-  }
-
-  /**
-   * Merge two regions. Asynchronous operation.
-   * @param nameofRegionsToMerge encoded or full name of daughter regions
-   * @param forcible true if do a compulsory merge, otherwise we will only merge
-   *          adjacent regions
-   */
-  @Override
-  public Future<Void> mergeRegionsAsync(final byte[][] nameofRegionsToMerge, final boolean forcible)
-      throws IOException {
-    Preconditions.checkArgument(nameofRegionsToMerge.length >= 2, "Can not merge only %s region",
-      nameofRegionsToMerge.length);
-    byte[][] encodedNameofRegionsToMerge = new byte[nameofRegionsToMerge.length][];
-    for (int i = 0; i < nameofRegionsToMerge.length; i++) {
-      encodedNameofRegionsToMerge[i] =
-        RegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ? nameofRegionsToMerge[i]
-          : Bytes.toBytes(RegionInfo.encodeRegionName(nameofRegionsToMerge[i]));
-    }
-
-    TableName tableName = null;
-    Pair<RegionInfo, ServerName> pair;
-
-    for(int i = 0; i < nameofRegionsToMerge.length; i++) {
-      pair = getRegion(nameofRegionsToMerge[i]);
-
-      if (pair != null) {
-        if (pair.getFirst().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
-          throw new IllegalArgumentException ("Can't invoke merge on non-default regions directly");
-        }
-        if (tableName == null) {
-          tableName = pair.getFirst().getTable();
-        } else  if (!tableName.equals(pair.getFirst().getTable())) {
-          throw new IllegalArgumentException ("Cannot merge regions from two different tables " +
-              tableName + " and " + pair.getFirst().getTable());
-        }
-      } else {
-        throw new UnknownRegionException (
-          "Can't invoke merge on unknown region "
-          + Bytes.toStringBinary(encodedNameofRegionsToMerge[i]));
-      }
-    }
-
-    MergeTableRegionsResponse response =
-        executeCallable(new MasterCallable<MergeTableRegionsResponse>(getConnection(),
-            getRpcControllerFactory()) {
-          Long nonceGroup = ng.getNonceGroup();
-          Long nonce = ng.newNonce();
-      @Override
-      protected MergeTableRegionsResponse rpcCall() throws Exception {
-        MergeTableRegionsRequest request = RequestConverter
-            .buildMergeTableRegionsRequest(
-                encodedNameofRegionsToMerge,
-                forcible,
-                nonceGroup,
-                nonce);
-        return master.mergeTableRegions(getRpcController(), request);
-      }
-    });
-    return new MergeTableRegionsFuture(this, tableName, response);
-  }
-
-  private static class MergeTableRegionsFuture extends TableFuture<Void> {
-    public MergeTableRegionsFuture(
-        final HBaseAdmin admin,
-        final TableName tableName,
-        final MergeTableRegionsResponse response) {
-      super(admin, tableName,
-          (response != null && response.hasProcId()) ? response.getProcId() : null);
-    }
-
-    public MergeTableRegionsFuture(
-        final HBaseAdmin admin,
-        final TableName tableName,
-        final Long procId) {
-      super(admin, tableName, procId);
-    }
-
-    @Override
-    public String getOperationType() {
-      return "MERGE_REGIONS";
-    }
-  }
-  /**
-   * Split one region. Synchronous operation.
-   * Note: It is not feasible to predict the length of split.
-   *   Therefore, this is for internal testing only.
-   * @param regionName encoded or full name of region
-   * @param splitPoint key where region splits
-   * @throws IOException if a remote or network exception occurs
-   */
-  @VisibleForTesting
-  public void splitRegionSync(byte[] regionName, byte[] splitPoint) throws IOException {
-    splitRegionSync(regionName, splitPoint, syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-
-  /**
-   * Split one region. Synchronous operation.
-   * @param regionName region to be split
-   * @param splitPoint split point
-   * @param timeout how long to wait on split
-   * @param units time units
-   * @throws IOException if a remote or network exception occurs
-   */
-  public void splitRegionSync(byte[] regionName, byte[] splitPoint, final long timeout,
-      final TimeUnit units) throws IOException {
-    get(splitRegionAsync(regionName, splitPoint), timeout, units);
-  }
-
-  @Override
-  public Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint)
-      throws IOException {
-    byte[] encodedNameofRegionToSplit = RegionInfo.isEncodedRegionName(regionName) ?
-        regionName : Bytes.toBytes(RegionInfo.encodeRegionName(regionName));
-    Pair<RegionInfo, ServerName> pair = getRegion(regionName);
-    if (pair != null) {
-      if (pair.getFirst() != null &&
-          pair.getFirst().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
-        throw new IllegalArgumentException ("Can't invoke split on non-default regions directly");
-      }
-    } else {
-      throw new UnknownRegionException(
-        "Can't invoke split on unknown region " + Bytes.toStringBinary(encodedNameofRegionToSplit));
-    }
-
-    return splitRegionAsync(pair.getFirst(), splitPoint);
-  }
-
-  Future<Void> splitRegionAsync(RegionInfo hri, byte[] splitPoint) throws IOException {
-    TableName tableName = hri.getTable();
-    if (hri.getStartKey() != null && splitPoint != null &&
-        Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
-      throw new IOException("should not give a splitkey which equals to startkey!");
-    }
-
-    SplitTableRegionResponse response = executeCallable(
-        new MasterCallable<SplitTableRegionResponse>(getConnection(), getRpcControllerFactory()) {
-          Long nonceGroup = ng.getNonceGroup();
-          Long nonce = ng.newNonce();
-          @Override
-          protected SplitTableRegionResponse rpcCall() throws Exception {
-            setPriority(tableName);
-            SplitTableRegionRequest request = RequestConverter
-                .buildSplitTableRegionRequest(hri, splitPoint, nonceGroup, nonce);
-            return master.splitRegion(getRpcController(), request);
-          }
-        });
-    return new SplitTableRegionFuture(this, tableName, response);
-  }
-
-  private static class SplitTableRegionFuture extends TableFuture<Void> {
-    public SplitTableRegionFuture(final HBaseAdmin admin,
-        final TableName tableName,
-        final SplitTableRegionResponse response) {
-      super(admin, tableName,
-          (response != null && response.hasProcId()) ? response.getProcId() : null);
-    }
-
-    public SplitTableRegionFuture(
-        final HBaseAdmin admin,
-        final TableName tableName,
-        final Long procId) {
-      super(admin, tableName, procId);
-    }
-
-    @Override
-    public String getOperationType() {
-      return "SPLIT_REGION";
-    }
-  }
-
-  @Override
-  public void split(final TableName tableName) throws IOException {
-    split(tableName, null);
-  }
-
-  @Override
-  public void split(final TableName tableName, final byte[] splitPoint) throws IOException {
-    checkTableExists(tableName);
-    for (HRegionLocation loc : connection.locateRegions(tableName, false, false)) {
-      ServerName sn = loc.getServerName();
-      if (sn == null) {
-        continue;
-      }
-      RegionInfo r = loc.getRegion();
-      // check for parents
-      if (r.isSplitParent()) {
-        continue;
-      }
-      // if a split point given, only split that particular region
-      if (r.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID ||
-          (splitPoint != null && !r.containsRow(splitPoint))) {
-        continue;
-      }
-      // call out to master to do split now
-      splitRegionAsync(r, splitPoint);
-    }
-  }
-
-  private static class ModifyTableFuture extends TableFuture<Void> {
-    public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName,
-        final ModifyTableResponse response) {
-      super(admin, tableName,
-          (response != null && response.hasProcId()) ? response.getProcId() : null);
-    }
-
-    public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName, final Long procId) {
-      super(admin, tableName, procId);
-    }
-
-    @Override
-    public String getOperationType() {
-      return "MODIFY";
-    }
-  }
-
-  /**
-   * @param regionName Name of a region.
-   * @return a pair of HRegionInfo and ServerName if <code>regionName</code> is
-   *  a verified region name (we call {@link
-   *  MetaTableAccessor#getRegionLocation(Connection, byte[])}
-   *  else null.
-   * Throw IllegalArgumentException if <code>regionName</code> is null.
-   * @throws IOException if a remote or network exception occurs
-   */
-  Pair<RegionInfo, ServerName> getRegion(final byte[] regionName) throws IOException {
-    if (regionName == null) {
-      throw new IllegalArgumentException("Pass a table name or region name");
-    }
-    Pair<RegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionName);
-    if (pair == null) {
-      final AtomicReference<Pair<RegionInfo, ServerName>> result = new AtomicReference<>(null);
-      final String encodedName = Bytes.toString(regionName);
-      MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
-        @Override
-        public boolean visit(Result data) throws IOException {
-          RegionInfo info = MetaTableAccessor.getRegionInfo(data);
-          if (info == null) {
-            LOG.warn("No serialized HRegionInfo in " + data);
-            return true;
-          }
-          RegionLocations rl = MetaTableAccessor.getRegionLocations(data);
-          boolean matched = false;
-          ServerName sn = null;
-          if (rl != null) {
-            for (HRegionLocation h : rl.getRegionLocations()) {
-              if (h != null && encodedName.equals(h.getRegion().getEncodedName())) {
-                sn = h.getServerName();
-                info = h.getRegion();
-                matched = true;
-              }
-            }
-          }
-          if (!matched) return true;
-          result.set(new Pair<>(info, sn));
-          return false; // found the region, stop
-        }
-      };
-
-      MetaTableAccessor.fullScanRegions(connection, visitor);
-      pair = result.get();
-    }
-    return pair;
-  }
-
-  /**
-   * If the input is a region name, it is returned as is. If it's an
-   * encoded region name, the corresponding region is found from meta
-   * and its region name is returned. If we can't find any region in
-   * meta matching the input as either region name or encoded region
-   * name, the input is returned as is. We don't throw unknown
-   * region exception.
-   */
-  private byte[] getRegionName(final byte[] regionNameOrEncodedRegionName) throws IOException {
-    if (Bytes.equals(regionNameOrEncodedRegionName,
-      RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()) ||
-      Bytes.equals(regionNameOrEncodedRegionName,
-        RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
-      return RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName();
-    }
-    byte[] tmp = regionNameOrEncodedRegionName;
-    Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
-    if (regionServerPair != null && regionServerPair.getFirst() != null) {
-      tmp = regionServerPair.getFirst().getRegionName();
-    }
-    return tmp;
-  }
-
-  /**
-   * Check if table exists or not
-   * @param tableName Name of a table.
-   * @return tableName instance
-   * @throws IOException if a remote or network exception occurs.
-   * @throws TableNotFoundException if table does not exist.
-   */
-  private TableName checkTableExists(final TableName tableName)
-      throws IOException {
-    return executeCallable(new RpcRetryingCallable<TableName>() {
-      @Override
-      protected TableName rpcCall(int callTimeout) throws Exception {
-        if (!MetaTableAccessor.tableExists(connection, tableName)) {
-          throw new TableNotFoundException(tableName);
-        }
-        return tableName;
-      }
-    });
-  }
-
-  @Override
-  public synchronized void shutdown() throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        setPriority(HConstants.HIGH_QOS);
-        master.shutdown(getRpcController(), ShutdownRequest.newBuilder().build());
-        return null;
-      }
-    });
-  }
-
-  @Override
-  public synchronized void stopMaster() throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        setPriority(HConstants.HIGH_QOS);
-        master.stopMaster(getRpcController(), StopMasterRequest.newBuilder().build());
-        return null;
-      }
-    });
-  }
-
-  @Override
-  public synchronized void stopRegionServer(final String hostnamePort)
-  throws IOException {
-    String hostname = Addressing.parseHostname(hostnamePort);
-    int port = Addressing.parsePort(hostnamePort);
-    final AdminService.BlockingInterface admin =
-      this.connection.getAdmin(ServerName.valueOf(hostname, port, 0));
-    // TODO: There is no timeout on this controller. Set one!
-    HBaseRpcController controller = rpcControllerFactory.newController();
-    controller.setPriority(HConstants.HIGH_QOS);
-    StopServerRequest request = RequestConverter.buildStopServerRequest(
-        "Called by admin client " + this.connection.toString());
-    try {
-      admin.stopServer(controller, request);
-    } catch (Exception e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
-  }
-
-  @Override
-  public boolean isMasterInMaintenanceMode() throws IOException {
-    return executeCallable(new MasterCallable<IsInMaintenanceModeResponse>(getConnection(),
-        this.rpcControllerFactory) {
-      @Override
-      protected IsInMaintenanceModeResponse rpcCall() throws Exception {
-        return master.isMasterInMaintenanceMode(getRpcController(),
-            IsInMaintenanceModeRequest.newBuilder().build());
-      }
-    }).getInMaintenanceMode();
-  }
-
-  @Override
-  public ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException {
-    return executeCallable(new MasterCallable<ClusterMetrics>(getConnection(),
-        this.rpcControllerFactory) {
-      @Override
-      protected ClusterMetrics rpcCall() throws Exception {
-        GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest(options);
-        return ClusterMetricsBuilder.toClusterMetrics(
-          master.getClusterStatus(getRpcController(), req).getClusterStatus());
-      }
-    });
-  }
-
-  @Override
-  public List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName)
-      throws IOException {
-    AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
-    HBaseRpcController controller = rpcControllerFactory.newController();
-    AdminProtos.GetRegionLoadRequest request =
-      RequestConverter.buildGetRegionLoadRequest(tableName);
-    try {
-      return admin.getRegionLoad(controller, request).getRegionLoadsList().stream()
-        .map(RegionMetricsBuilder::toRegionMetrics).collect(Collectors.toList());
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
-  }
-
-  @Override
-  public Configuration getConfiguration() {
-    return this.conf;
-  }
-
-  /**
-   * Do a get with a timeout against the passed in <code>future</code>.
-   */
-  private static <T> T get(final Future<T> future, final long timeout, final TimeUnit units)
-  throws IOException {
-    try {
-      // TODO: how long should we wait? Spin forever?
-      return future.get(timeout, units);
-    } catch (InterruptedException e) {
-      IOException ioe = new InterruptedIOException("Interrupt while waiting on " + future);
-      ioe.initCause(e);
-      throw ioe;
-    } catch (TimeoutException e) {
-      throw new TimeoutIOException(e);
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof IOException) {
-        throw (IOException)e.getCause();
-      } else {
-        throw new IOException(e.getCause());
-      }
-    }
-  }
-
-  @Override
-  public Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor)
-      throws IOException {
-    CreateNamespaceResponse response =
-        executeCallable(new MasterCallable<CreateNamespaceResponse>(getConnection(),
-            getRpcControllerFactory()) {
-      @Override
-      protected CreateNamespaceResponse rpcCall() throws Exception {
-        return master.createNamespace(getRpcController(),
-          CreateNamespaceRequest.newBuilder().setNamespaceDescriptor(ProtobufUtil.
-              toProtoNamespaceDescriptor(descriptor)).build());
-      }
-    });
-    return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
-      @Override
-      public String getOperationType() {
-        return "CREATE_NAMESPACE";
-      }
-    };
-  }
-
-  @Override
-  public Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor)
-      throws IOException {
-    ModifyNamespaceResponse response =
-        executeCallable(new MasterCallable<ModifyNamespaceResponse>(getConnection(),
-            getRpcControllerFactory()) {
-      @Override
-      protected ModifyNamespaceResponse rpcCall() throws Exception {
-        // TODO: set priority based on NS?
-        return master.modifyNamespace(getRpcController(), ModifyNamespaceRequest.newBuilder().
-          setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
-       }
-    });
-    return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
-      @Override
-      public String getOperationType() {
-        return "MODIFY_NAMESPACE";
-      }
-    };
-  }
-
-  @Override
-  public Future<Void> deleteNamespaceAsync(final String name)
-      throws IOException {
-    DeleteNamespaceResponse response =
-        executeCallable(new MasterCallable<DeleteNamespaceResponse>(getConnection(),
-            getRpcControllerFactory()) {
-      @Override
-      protected DeleteNamespaceResponse rpcCall() throws Exception {
-        // TODO: set priority based on NS?
-        return master.deleteNamespace(getRpcController(), DeleteNamespaceRequest.newBuilder().
-          setNamespaceName(name).build());
-        }
-      });
-    return new NamespaceFuture(this, name, response.getProcId()) {
-      @Override
-      public String getOperationType() {
-        return "DELETE_NAMESPACE";
-      }
-    };
-  }
-
-  @Override
-  public NamespaceDescriptor getNamespaceDescriptor(final String name)
-      throws NamespaceNotFoundException, IOException {
-    return executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected NamespaceDescriptor rpcCall() throws Exception {
-        return ProtobufUtil.toNamespaceDescriptor(
-            master.getNamespaceDescriptor(getRpcController(),
-                GetNamespaceDescriptorRequest.newBuilder().
-                  setNamespaceName(name).build()).getNamespaceDescriptor());
-      }
-    });
-  }
-
-  /**
-   * List available namespaces
-   * @return List of namespace names
-   * @throws IOException if a remote or network exception occurs
-   */
-  @Override
-  public String[] listNamespaces() throws IOException {
-    return executeCallable(new MasterCallable<String[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected String[] rpcCall() throws Exception {
-        List<String> list = master.listNamespaces(getRpcController(),
-          ListNamespacesRequest.newBuilder().build()).getNamespaceNameList();
-        return list.toArray(new String[list.size()]);
-      }
-    });
-  }
-
-  /**
-   * List available namespace descriptors
-   * @return List of descriptors
-   * @throws IOException if a remote or network exception occurs
-   */
-  @Override
-  public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
-    return executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected NamespaceDescriptor[] rpcCall() throws Exception {
-        List<HBaseProtos.NamespaceDescriptor> list =
-            master.listNamespaceDescriptors(getRpcController(),
-              ListNamespaceDescriptorsRequest.newBuilder().build()).getNamespaceDescriptorList();
-        NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
-        for(int i = 0; i < list.size(); i++) {
-          res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
-        }
-        return res;
-      }
-    });
-  }
-
-  @Override
-  public String getProcedures() throws IOException {
-    return executeCallable(new MasterCallable<String>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected String rpcCall() throws Exception {
-        GetProceduresRequest request = GetProceduresRequest.newBuilder().build();
-        GetProceduresResponse response = master.getProcedures(getRpcController(), request);
-        return ProtobufUtil.toProcedureJson(response.getProcedureList());
-      }
-    });
-  }
-
-  @Override
-  public String getLocks() throws IOException {
-    return executeCallable(new MasterCallable<String>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected String rpcCall() throws Exception {
-        GetLocksRequest request = GetLocksRequest.newBuilder().build();
-        GetLocksResponse response = master.getLocks(getRpcController(), request);
-        return ProtobufUtil.toLockJson(response.getLockList());
-      }
-    });
-  }
-
-  @Override
-  public TableName[] listTableNamesByNamespace(final String name) throws IOException {
-    return executeCallable(new MasterCallable<TableName[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected TableName[] rpcCall() throws Exception {
-        List<HBaseProtos.TableName> tableNames =
-            master.listTableNamesByNamespace(getRpcController(), ListTableNamesByNamespaceRequest.
-                newBuilder().setNamespaceName(name).build())
-            .getTableNameList();
-        TableName[] result = new TableName[tableNames.size()];
-        for (int i = 0; i < tableNames.size(); i++) {
-          result[i] = ProtobufUtil.toTableName(tableNames.get(i));
-        }
-        return result;
-      }
-    });
-  }
-
-  /**
-   * Is HBase available? Throw an exception if not.
-   * <p/>
-   * TODO: do not expose ZKConnectionException.
-   * @param conf system configuration
-   * @throws MasterNotRunningException if the master is not running.
-   * @throws ZooKeeperConnectionException if unable to connect to zookeeper.
-   */
-  public static void available(final Configuration conf)
-      throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
-    Configuration copyOfConf = HBaseConfiguration.create(conf);
-    // We set it to make it fail as soon as possible if HBase is not available
-    copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
-    copyOfConf.setInt("zookeeper.recovery.retry", 0);
-
-    // Check ZK first.
-    // If the connection exists, we may have a connection to ZK that does not work anymore
-    try (ConnectionImplementation connection = ConnectionFactory.createConnectionImpl(copyOfConf,
-      null, UserProvider.instantiate(copyOfConf).getCurrent())) {
-      // can throw MasterNotRunningException
-      connection.isMasterRunning();
-    }
-  }
-
-  private RollWALWriterResponse rollWALWriterImpl(final ServerName sn) throws IOException,
-      FailedLogCloseException {
-    final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
-    // TODO: There is no timeout on this controller. Set one!
-    HBaseRpcController controller = rpcControllerFactory.newController();
-    try {
-      return admin.rollWALWriter(controller, request);
-    } catch (ServiceException e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
-  }
-
-  @Override
-  public synchronized void rollWALWriter(ServerName serverName)
-      throws IOException, FailedLogCloseException {
-    rollWALWriterImpl(serverName);
-  }
-
-  @Override
-  public CompactionState getCompactionState(final TableName tableName)
-  throws IOException {
-    return getCompactionState(tableName, CompactType.NORMAL);
-  }
-
-  @Override
-  public CompactionState getCompactionStateForRegion(final byte[] regionName)
-  throws IOException {
-    final Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
-    if (regionServerPair == null) {
-      throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
-    }
-    if (regionServerPair.getSecond() == null) {
-      throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
-    }
-    ServerName sn = regionServerPair.getSecond();
-    final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    // TODO: There is no timeout on this controller. Set one!
-    HBaseRpcController controller = rpcControllerFactory.newController();
-    GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
-      regionServerPair.getFirst().getRegionName(), true);
-    GetRegionInfoResponse response;
-    try {
-      response = admin.getRegionInfo(controller, request);
-    } catch (ServiceException e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
-    if (response.getCompactionState() != null) {
-      return ProtobufUtil.createCompactionState(response.getCompactionState());
-    }
-    return null;
-  }
-
-  @Override
-  public void snapshot(SnapshotDescription snapshotDesc)
-      throws IOException, SnapshotCreationException, IllegalArgumentException {
-    // actually take the snapshot
-    SnapshotProtos.SnapshotDescription snapshot =
-      ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc);
-    SnapshotResponse response = asyncSnapshot(snapshot);
-    final IsSnapshotDoneRequest request =
-        IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build();
-    IsSnapshotDoneResponse done = null;
-    long start = EnvironmentEdgeManager.currentTime();
-    long max = response.getExpectedTimeout();
-    long maxPauseTime = max / this.numRetries;
-    int tries = 0;
-    LOG.debug("Waiting a max of " + max + " ms for snapshot '" +
-        ClientSnapshotDescriptionUtils.toString(snapshot) + "'' to complete. (max " +
-        maxPauseTime + " ms per retry)");
-    while (tries == 0
-        || ((EnvironmentEdgeManager.currentTime() - start) < max && !done.getDone())) {
-      try {
-        // sleep a backoff <= pauseTime amount
-        long sleep = getPauseTime(tries++);
-        sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
-        LOG.debug("(#" + tries + ") Sleeping: " + sleep +
-          "ms while waiting for snapshot completion.");
-        Thread.sleep(sleep);
-      } catch (InterruptedException e) {
-        throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
-      }
-      LOG.debug("Getting current status of snapshot from master...");
-      done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection(),
-          getRpcControllerFactory()) {
-        @Override
-        protected IsSnapshotDoneResponse rpcCall() throws Exception {
-          return master.isSnapshotDone(getRpcController(), request);
-        }
-      });
-    }
-    if (!done.getDone()) {
-      throw new SnapshotCreationException("Snapshot '" + snapshot.getName()
-          + "' wasn't completed in expectedTime:" + max + " ms", snapshotDesc);
-    }
-  }
-
-  @Override
-  public Future<Void> snapshotAsync(SnapshotDescription snapshotDesc)
-      throws IOException, SnapshotCreationException {
-    asyncSnapshot(ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc));
-    return new ProcedureFuture<Void>(this, null) {
-
-      @Override
-      protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutException {
-        waitForState(deadlineTs, new WaitForStateCallable() {
-
-          @Override
-          public void throwInterruptedException() throws InterruptedIOException {
-            throw new InterruptedIOException(
-              "Interrupted while waiting for taking snapshot" + snapshotDesc);
-          }
-
-          @Override
-          public void throwTimeoutException(long elapsedTime) throws TimeoutException {
-            throw new TimeoutException("Snapshot '" + snapshotDesc.getName() +
-              "' wasn't completed in expectedTime:" + elapsedTime + " ms");
-          }
-
-          @Override
-          public boolean checkState(int tries) throws IOException {
-            return isSnapshotFinished(snapshotDesc);
-          }
-        });
-        return null;
-      }
-    };
-  }
-
-  private SnapshotResponse asyncSnapshot(SnapshotProtos.SnapshotDescription snapshot)
-      throws IOException {
-    ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
-    final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot)
-        .build();
-    // run the snapshot on the master
-    return executeCallable(new MasterCallable<SnapshotResponse>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected SnapshotResponse rpcCall() throws Exception {
-        return master.snapshot(getRpcController(), request);
-      }
-    });
-  }
-
-  @Override
-  public boolean isSnapshotFinished(final SnapshotDescription snapshotDesc)
-      throws IOException, HBaseSnapshotException, UnknownSnapshotException {
-    final SnapshotProtos.SnapshotDescription snapshot =
-        ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc);
-    return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected IsSnapshotDoneResponse rpcCall() throws Exception {
-        return master.isSnapshotDone(getRpcController(),
-          IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build());
-      }
-    }).getDone();
-  }
-
-  @Override
-  public void restoreSnapshot(final String snapshotName)
-      throws IOException, RestoreSnapshotException {
-    boolean takeFailSafeSnapshot =
-        conf.getBoolean(HConstants.SNAPSHOT_RESTORE_TAKE_FAILSAFE_SNAPSHOT,
-          HConstants.DEFAULT_SNAPSHOT_RESTORE_TAKE_FAILSAFE_SNAPSHOT);
-    restoreSnapshot(snapshotName, takeFailSafeSnapshot);
-  }
-
-  /**
-   * Check whether the snapshot exists and contains disabled table
-   *
-   * @param snapshotName name of the snapshot to restore
-   * @throws IOException if a remote or network exception occurs
-   * @throws RestoreSnapshotException if no valid snapshot is found
-   */
-  private TableName getTableNameBeforeRestoreSnapshot(final String snapshotName)
-      throws IOException, RestoreSnapshotException {
-    TableName tableName = null;
-    for (SnapshotDescription snapshotInfo: listSnapshots()) {
-      if (snapshotInfo.getName().equals(snapshotName)) {
-        tableName = snapshotInfo.getTableName();
-        break;
-      }
-    }
-
-    if (tableName == null) {
-      throw new RestoreSnapshotException(
-        "Unable to find the table name for snapshot=" + snapshotName);
-    }
-    return tableName;
-  }
-
-  @Override
-  public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot)
-      throws IOException, RestoreSnapshotException {
-    restoreSnapshot(snapshotName, takeFailSafeSnapshot, false);
-  }
-
-  @Override
-  public void restoreSnapshot(final String snapshotName, final boolean takeFailSafeSnapshot,
-      final boolean restoreAcl) throws IOException, RestoreSnapshotException {
-    TableName tableName = getTableNameBeforeRestoreSnapshot(snapshotName);
-
-    // The table does not exists, switch to clone.
-    if (!tableExists(tableName)) {
-      cloneSnapshot(snapshotName, tableName, restoreAcl);
-      return;
-    }
-
-    // Check if the table is disabled
-    if (!isTableDisabled(tableName)) {
-      throw new TableNotDisabledException(tableName);
-    }
-
-    // Take a snapshot of the current state
-    String failSafeSnapshotSnapshotName = null;
-    if (takeFailSafeSnapshot) {
-      failSafeSnapshotSnapshotName = conf.get("hbase.snapshot.restore.failsafe.name",
-        "hbase-failsafe-{snapshot.name}-{restore.timestamp}");
-      failSafeSnapshotSnapshotName = failSafeSnapshotSnapshotName
-        .replace("{snapshot.name}", snapshotName)
-        .replace("{table.name}", tableName.toString().replace(TableName.NAMESPACE_DELIM, '.'))
-        .replace("{restore.timestamp}", String.valueOf(EnvironmentEdgeManager.currentTime()));
-      LOG.info("Taking restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
-      snapshot(failSafeSnapshotSnapshotName, tableName);
-    }
-
-    try {
-      // Restore snapshot
-      get(
-        internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl),
-        syncWaitTimeout,
-        TimeUnit.MILLISECONDS);
-    } catch (IOException e) {
-      // Something went wrong during the restore...
-      // if the pre-restore snapshot is available try to rollback
-      if (takeFailSafeSnapshot) {
-        try {
-          get(
-            internalRestoreSnapshotAsync(failSafeSnapshotSnapshotName, tableName, restoreAcl),
-            syncWaitTimeout,
-            TimeUnit.MILLISECONDS);
-          String msg = "Restore snapshot=" + snapshotName +
-            " failed. Rollback to snapshot=" + failSafeSnapshotSnapshotName + " succeeded.";
-          LOG.error(msg, e);
-          throw new RestoreSnapshotException(msg, e);
-        } catch (IOException ex) {
-          String msg = "Failed to restore and rollback to snapshot=" + failSafeSnapshotSnapshotName;
-          LOG.error(msg, ex);
-          throw new RestoreSnapshotException(msg, e);
-        }
-      } else {
-        throw new RestoreSnapshotException("Failed to restore snapshot=" + snapshotName, e);
-      }
-    }
-
-    // If the restore is succeeded, delete the pre-restore snapshot
-    if (takeFailSafeSnapshot) {
-      try {
-        LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
-        deleteSnapshot(failSafeSnapshotSnapshotName);
-      } catch (IOException e) {
-        LOG.error("Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName, e);
-      }
-    }
-  }
-
-  @Override
-  public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
-      boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException {
-    if (tableExists(tableName)) {
-      throw new TableExistsException(tableName);
-    }
-    return internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl);
-  }
-
-  @Override
-  public byte[] execProcedureWithReturn(String signature, String instance, Map<String,
-      String> props) throws IOException {
-    ProcedureDescription desc = ProtobufUtil.buildProcedureDescription(signature, instance, props);
-    final ExecProcedureRequest request =
-        ExecProcedureRequest.newBuilder().setProcedure(desc).build();
-    // run the procedure on the master
-    ExecProcedureResponse response = executeCallable(
-      new MasterCallable<ExecProcedureResponse>(getConnection(), getRpcControllerFactory()) {
-        @Override
-        protected ExecProcedureResponse rpcCall() throws Exception {
-          return master.execProcedureWithRet(getRpcController(), request);
-        }
-      });
-
-    return response.hasReturnData() ? response.getReturnData().toByteArray() : null;
-  }
-
-  @Override
-  public void execProcedure(String signature, String instance, Map<String, String> props)
-      throws IOException {
-    ProcedureDescription desc = ProtobufUtil.buildProcedureDescription(signature, instance, props);
-    final ExecProcedureRequest request =
-        ExecProcedureRequest.newBuilder().setProcedure(desc).build();
-    // run the procedure on the master
-    ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
-        getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected ExecProcedureResponse rpcCall() throws Exception {
-        return master.execProcedure(getRpcController(), request);
-      }
-    });
-
-    long start = EnvironmentEdgeManager.currentTime();
-    long max = response.getExpectedTimeout();
-    long maxPauseTime = max / this.numRetries;
-    int tries = 0;
-    LOG.debug("Waiting a max of " + max + " ms for procedure '" +
-        signature + " : " + instance + "'' to complete. (max " + maxPauseTime + " ms per retry)");
-    boolean done = false;
-    while (tries == 0
-        || ((EnvironmentEdgeManager.currentTime() - start) < max && !done)) {
-      try {
-        // sleep a backoff <= pauseTime amount
-        long sleep = getPauseTime(tries++);
-        sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
-        LOG.debug("(#" + tries + ") Sleeping: " + sleep +
-          "ms while waiting for procedure completion.");
-        Thread.sleep(sleep);
-      } catch (InterruptedException e) {
-        throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
-      }
-      LOG.debug("Getting current status of procedure from master...");
-      done = isProcedureFinished(signature, instance, props);
-    }
-    if (!done) {
-      throw new IOException("Procedure '" + signature + " : " + instance
-          + "' wasn't completed in expectedTime:" + max + " ms");
-    }
-  }
-
-  @Override
-  public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
-      throws IOException {
-    ProcedureDescription desc = ProtobufUtil.buildProcedureDescription(signature, instance, props);
-    return executeCallable(
-      new MasterCallable<IsProcedureDoneResponse>(getConnection(), getRpcControllerFactory()) {
-        @Override
-        protected IsProcedureDoneResponse rpcCall() throws Exception {
-          return master.isProcedureDone(getRpcController(),
-            IsProcedureDoneRequest.newBuilder().setProcedure(desc).build());
-        }
-      }).getDone();
-  }
-
-  /**
-   * Execute Restore/Clone snapshot and wait for the server to complete (blocking).
-   * To check if the cloned table exists, use {@link #isTableAvailable} -- it is not safe to
-   * create an HTable instance to this table before it is available.
-   * @param snapshotName snapshot to restore
-   * @param tableName table name to restore the snapshot on
-   * @throws IOException if a remote or network exception occurs
-   * @throws RestoreSnapshotException if snapshot failed to be restored
-   * @throws IllegalArgumentException if the restore request is formatted incorrectly
-   */
-  private Future<Void> internalRestoreSnapshotAsync(final String snapshotName,
-      final TableName tableName, final boolean restoreAcl)
-      throws IOException, RestoreSnapshotException {
-    final SnapshotProtos.SnapshotDescription snapshot =
-        SnapshotProtos.SnapshotDescription.newBuilder()
-        .setName(snapshotName).setTable(tableName.getNameAsString()).build();
-
-    // actually restore the snapshot
-    ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
-
-    RestoreSnapshotResponse response = executeCallable(
-        new MasterCallable<RestoreSnapshotResponse>(getConnection(), getRpcControllerFactory()) {
-          Long nonceGroup = ng.getNonceGroup();
-          Long nonce = ng.newNonce();
-      @Override
-      protected RestoreSnapshotResponse rpcCall() throws Exception {
-        final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder()
-            .setSnapshot(snapshot)
-            .setNonceGroup(nonceGroup)
-            .setNonce(nonce)
-            .setRestoreACL(restoreAcl)
-            .build();
-        return master.restoreSnapshot(getRpcController(), request);
-      }
-    });
-
-    return new RestoreSnapshotFuture(this, snapshot, tableName, response);
-  }
-
-  private static class RestoreSnapshotFuture extends TableFuture<Void> {
-    public RestoreSnapshotFuture(
-        final HBaseAdmin admin,
-        final SnapshotProtos.SnapshotDescription snapshot,
-        final TableName tableName,
-        final RestoreSnapshotResponse response) {
-      super(admin, tableName,
-          (response != null && response.hasProcId()) ? response.getProcId() : null);
-
-      if (response != null && !response.hasProcId()) {
-        throw new UnsupportedOperationException("Client could not call old version of Server");
-      }
-    }
-
-    public RestoreSnapshotFuture(
-        final HBaseAdmin admin,
-        final TableName tableName,
-        final Long procId) {
-      super(admin, tableName, procId);
-    }
-
-    @Override
-    public String getOperationType() {
-      return "MODIFY";
-    }
-  }
-
-  @Override
-  public List<SnapshotDescription> listSnapshots() throws IOException {
-    return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected List<SnapshotDescription> rpcCall() throws Exception {
-        List<SnapshotProtos.SnapshotDescription> snapshotsList = master
-            .getCompletedSnapshots(getRpcController(),
-                GetCompletedSnapshotsRequest.newBuilder().build())
-            .getSnapshotsList();
-        List<SnapshotDescription> result = new ArrayList<>(snapshotsList.size());
-        for (SnapshotProtos.SnapshotDescription snapshot : snapshotsList) {
-          result.add(ProtobufUtil.createSnapshotDesc(snapshot));
-        }
-        return result;
-      }
-    });
-  }
-
-  @Override
-  public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
-    List<SnapshotDescription> matched = new LinkedList<>();
-    List<SnapshotDescription> snapshots = listSnapshots();
-    for (SnapshotDescription snapshot : snapshots) {
-      if (pattern.matcher(snapshot.getName()).matches()) {
-        matched.add(snapshot);
-      }
-    }
-    return matched;
-  }
-
-  @Override
-  public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
-      Pattern snapshotNamePattern) throws IOException {
-    TableName[] tableNames = listTableNames(tableNamePattern);
-
-    List<SnapshotDescription> tableSnapshots = new LinkedList<>();
-    List<SnapshotDescription> snapshots = listSnapshots(snapshotNamePattern);
-
-    List<TableName> listOfTableNames = Arrays.asList(tableNames);
-    for (SnapshotDescription snapshot : snapshots) {
-      if (listOfTableNames.contains(snapshot.getTableName())) {
-        tableSnapshots.add(snapshot);
-      }
-    }
-    return tableSnapshots;
-  }
-
-  @Override
-  public void deleteSnapshot(final String snapshotName) throws IOException {
-    // make sure the snapshot is possibly valid
-    TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName));
-    // do the delete
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.deleteSnapshot(getRpcController(),
-          DeleteSnapshotRequest.newBuilder().setSnapshot(
-                SnapshotProtos.SnapshotDescription.newBuilder().setName(snapshotName).build())
-              .build()
-        );
-        return null;
-      }
-    });
-  }
-
-  @Override
-  public void deleteSnapshots(final Pattern pattern) throws IOException {
-    List<SnapshotDescription> snapshots = listSnapshots(pattern);
-    for (final SnapshotDescription snapshot : snapshots) {
-      try {
-        internalDeleteSnapshot(snapshot);
-      } catch (IOException ex) {
-        LOG.info("Failed to delete snapshot " + snapshot.getName() + " for table "
-                + snapshot.getTableNameAsString(), ex);
-      }
-    }
-  }
-
-  private void internalDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        this.master.deleteSnapshot(getRpcController(), DeleteSnapshotRequest.newBuilder()
-          .setSnapshot(ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot)).build());
-        return null;
-      }
-    });
-  }
-
-  @Override
-  public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
-      throws IOException {
-    List<SnapshotDescription> snapshots = listTableSnapshots(tableNamePattern, snapshotNamePattern);
-    for (SnapshotDescription snapshot : snapshots) {
-      try {
-        internalDeleteSnapshot(snapshot);
-        LOG.debug("Successfully deleted snapshot: " + snapshot.getName());
-      } catch (IOException e) {
-        LOG.error("Failed to delete snapshot: " + snapshot.getName(), e);
-      }
-    }
-  }
-
-  @Override
-  public void setQuota(final QuotaSettings quota) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        this.master.setQuota(getRpcController(), QuotaSettings.buildSetQuotaRequestProto(quota));
-        return null;
-      }
-    });
-  }
-
-  @Override
-  public List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException {
-    List<QuotaSettings> quotas = new LinkedList<>();
-    try (QuotaRetriever retriever = QuotaRetriever.open(conf, filter)) {
-      Iterator<QuotaSettings> iterator = retriever.iterator();
-      while (iterator.hasNext()) {
-        quotas.add(iterator.next());
-      }
-    }
-    return quotas;
-  }
-
-  private <C extends RetryingCallable<V> & Closeable, V> V executeCallable(C callable)
-      throws IOException {
-    return executeCallable(callable, rpcCallerFactory, operationTimeout, rpcTimeout);
-  }
-
-  static private <C extends RetryingCallable<V> & Closeable, V> V executeCallable(C callable,
-             RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout, int rpcTimeout)
-  throws IOException {
-    RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller(rpcTimeout);
-    try {
-      return caller.callWithRetries(callable, operationTimeout);
-    } finally {
-      callable.close();
-    }
-  }
-
-  @Override
-  // Coprocessor Endpoint against the Master.
-  public CoprocessorRpcChannel coprocessorService() {
-    return new SyncCoprocessorRpcChannel() {
-      @Override
-      protected Message callExecService(final RpcController controller,
-          final Descriptors.MethodDescriptor method, final Message request,
-          final Message responsePrototype)
-      throws IOException {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Call: " + method.getName() + ", " + request.toString());
-        }
-        // Try-with-resources so close gets called when we are done.
-        try (MasterCallable<CoprocessorServiceResponse> callable =
-            new MasterCallable<CoprocessorServiceResponse>(connection,
-                connection.getRpcControllerFactory()) {
-          @Override
-          protected CoprocessorServiceResponse rpcCall() throws Exception {
-            CoprocessorServiceRequest csr =
-                CoprocessorRpcUtils.getCoprocessorServiceRequest(method, request);
-            return this.master.execMasterService(getRpcController(), csr);
-          }
-        }) {
-          // TODO: Are we retrying here? Does not seem so. We should use RetryingRpcCaller
-          callable.prepare(false);
-          int operationTimeout = connection.getConnectionConfiguration().getOperationTimeout();
-          CoprocessorServiceResponse result = callable.call(operationTimeout);
-          return CoprocessorRpcUtils.getResponse(result, responsePrototype);
-        }
-      }
-    };
-  }
-
-  @Override
-  public CoprocessorRpcChannel coprocessorService(final ServerName serverName) {
-    return new SyncCoprocessorRpcChannel() {
-      @Override
-      protected Message callExecService(RpcController controller,
-          Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
-      throws IOException {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Call: " + method.getName() + ", " + request.toString());
-        }
-        CoprocessorServiceRequest csr =
-            CoprocessorRpcUtils.getCoprocessorServiceRequest(method, request);
-        // TODO: Are we retrying here? Does not seem so. We should use RetryingRpcCaller
-        // TODO: Make this same as RegionCoprocessorRpcChannel and MasterCoprocessorRpcChannel. They
-        // are all different though should do same thing; e.g. RpcChannel setup.
-        ClientProtos.ClientService.BlockingInterface stub = connection.getClient(serverName);
-        CoprocessorServiceResponse result;
-        try {
-          result = stub.
-              execRegionServerService(connection.getRpcControllerFactory().newController(), csr);
-          return CoprocessorRpcUtils.getResponse(result, responsePrototype);
-        } catch (ServiceException e) {
-          throw ProtobufUtil.handleRemoteException(e);
-        }
-      }
-    };
-  }
-
-  @Override
-  public void updateConfiguration(final ServerName server) throws IOException {
-    final AdminService.BlockingInterface admin = this.connection.getAdmin(server);
-    Callable<Void> callable = new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        admin.updateConfiguration(null, UpdateConfigurationRequest.getDefaultInstance());
-        return null;
-      }
-    };
-    ProtobufUtil.call(callable);
-  }
-
-  @Override
-  public void updateConfiguration() throws IOException {
-    ClusterMetrics status = getClusterMetrics(
-      EnumSet.of(Option.LIVE_SERVERS, Option.MASTER, Option.BACKUP_MASTERS));
-    for (ServerName server : status.getLiveServerMetrics().keySet()) {
-      updateConfiguration(server);
-    }
-
-    updateConfiguration(status.getMasterName());
-
-    for (ServerName server : status.getBackupMasterNames()) {
-      updateConfiguration(server);
-    }
-  }
-
-  @Override
-  public long getLastMajorCompactionTimestamp(final TableName tableName) throws IOException {
-    return executeCallable(new MasterCallable<Long>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Long rpcCall() throws Exception {
-        MajorCompactionTimestampRequest req =
-            MajorCompactionTimestampRequest.newBuilder()
-                .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
-        return master.getLastMajorCompactionTimestamp(getRpcController(), req).
-            getCompactionTimestamp();
-      }
-    });
-  }
-
-  @Override
-  public long getLastMajorCompactionTimestampForRegion(final byte[] regionName) throws IOException {
-    return executeCallable(new MasterCallable<Long>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Long rpcCall() throws Exception {
-        MajorCompactionTimestampForRegionRequest req =
-            MajorCompactionTimestampForRegionRequest.newBuilder().setRegion(RequestConverter
-                      .buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName)).build();
-        return master.getLastMajorCompactionTimestampForRegion(getRpcController(), req)
-            .getCompactionTimestamp();
-      }
-    });
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void compact(final TableName tableName, final byte[] columnFamily, CompactType compactType)
-    throws IOException, InterruptedException {
-    compact(tableName, columnFamily, false, compactType);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void compact(final TableName tableName, CompactType compactType)
-    throws IOException, InterruptedException {
-    compact(tableName, null, false, compactType);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void majorCompact(final TableName tableName, final byte[] columnFamily,
-    CompactType compactType) throws IOException, InterruptedException {
-    compact(tableName, columnFamily, true, compactType);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void majorCompact(final TableName tableName, CompactType compactType)
-          throws IOException, InterruptedException {
-    compact(tableName, null, true, compactType);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public CompactionState getCompactionState(final TableName tableName, CompactType compactType)
-      throws IOException {
-    checkTableExists(tableName);
-    if (!isTableEnabled(tableName)) {
-      // If the table is disabled, the compaction state of the table should always be NONE
-      return ProtobufUtil.createCompactionState(
-        AdminProtos.GetRegionInfoResponse.CompactionState.NONE);
-    }
-
-    AdminProtos.GetRegionInfoResponse.CompactionState state =
-      AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
-
-    // TODO: There is no timeout on this controller. Set one!
-    HBaseRpcController rpcController = rpcControllerFactory.newController();
-    switch (compactType) {
-      case MOB:
-        final AdminProtos.AdminService.BlockingInterface masterAdmin =
-          this.connection.getAdminForMaster();
-        Callable<AdminProtos.GetRegionInfoResponse.CompactionState> callable =
-          new Callable<AdminProtos.GetRegionInfoResponse.CompactionState>() {
-            @Override
-            public AdminProtos.GetRegionInfoResponse.CompactionState call() throws Exception {
-              RegionInfo info = RegionInfo.createMobRegionInfo(tableName);
-              GetRegionInfoRequest request =
-                RequestConverter.buildGetRegionInfoRequest(info.getRegionName(), true);
-              GetRegionInfoResponse response = masterAdmin.getRegionInfo(rpcController, request);
-              return response.getCompactionState();
-            }
-          };
-        state = ProtobufUtil.call(callable);
-        break;
-      case NORMAL:
-        for (HRegionLocation loc : connection.locateRegions(tableName, false, false)) {
-          ServerName sn = loc.getServerName();
-          if (sn == null) {
-            continue;
-          }
-          byte[] regionName = loc.getRegion().getRegionName();
-          AdminService.BlockingInterface snAdmin = this.connection.getAdmin(sn);
-          try {
-            Callable<GetRegionInfoResponse> regionInfoCallable =
-              new Callable<GetRegionInfoResponse>() {
-                @Override
-                public GetRegionInfoResponse call() throws Exception {
-                  GetRegionInfoRequest request =
-                    RequestConverter.buildGetRegionInfoRequest(regionName, true);
-                  return snAdmin.getRegionInfo(rpcController, request);
-                }
-              };
-            GetRegionInfoResponse response = ProtobufUtil.call(regionInfoCallable);
-            switch (response.getCompactionState()) {
-              case MAJOR_AND_MINOR:
-                return CompactionState.MAJOR_AND_MINOR;
-              case MAJOR:
-                if (state == AdminProtos.GetRegionInfoResponse.CompactionState.MINOR) {
-                  return CompactionState.MAJOR_AND_MINOR;
-                }
-                state = AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR;
-                break;
-              case MINOR:
-                if (state == AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR) {
-                  return CompactionState.MAJOR_AND_MINOR;
-                }
-                state = AdminProtos.GetRegionInfoResponse.CompactionState.MINOR;
-                break;
-              case NONE:
-              default: // nothing, continue
-            }
-          } catch (NotServingRegionException e) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Trying to get compaction state of " + loc.getRegion() + ": " +
-                StringUtils.stringifyException(e));
-            }
-          } catch (RemoteException e) {
-            if (e.getMessage().indexOf(NotServingRegionException.class.getName()) >= 0) {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Trying to get compaction state of " + loc.getRegion() + ": " +
-                  StringUtils.stringifyException(e));
-              }
-            } else {
-              throw e;
-            }
-          }
-        }
-        break;
-      default:
-        throw new IllegalArgumentException("Unknown compactType: " + compactType);
-    }
-    if (state != null) {
-      return ProtobufUtil.createCompactionState(state);
-    }
-    return null;
-  }
-
-  /**
-   * Future that waits on a procedure result.
-   * Returned by the async version of the Admin calls,
-   * and used internally by the sync calls to wait on the result of the procedure.
-   */
-  @InterfaceAudience.Private
-  @InterfaceStability.Evolving
-  protected static class ProcedureFuture<V> implements Future<V> {
-    private ExecutionException exception = null;
-    private boolean procResultFound = false;
-    private boolean done = false;
-    private boolean cancelled = false;
-    private V result = null;
-
-    private final HBaseAdmin admin;
-    protected final Long procId;
-
-    public ProcedureFuture(final HBaseAdmin admin, final Long procId) {
-      this.admin = admin;
-      this.procId = procId;
-    }
-
-    @Override
-    public boolean cancel(boolean mayInterruptIfRunning) {
-      AbortProcedureRequest abortProcRequest = AbortProcedureRequest.newBuilder()
-          .setProcId(procId).setMayInterruptIfRunning(mayInterruptIfRunning).build();
-      try {
-        cancelled = abortProcedureResult(abortProcRequest).getIsProcedureAborted();
-        if (cancelled) {
-          done = true;
-        }
-      } catch (IOException e) {
-        // Cancell thrown exception for some reason. At this time, we are not sure whether
-        // the cancell succeeds or fails. We assume that it is failed, but print out a warning
-        // for debugging purpose.
-        LOG.warn(
-          "Cancelling the procedure with procId=" + procId + " throws exception " + e.getMessage(),
-          e);
-        cancelled = false;
-      }
-      return cancelled;
-    }
-
-    @Override
-    public boolean isCancelled() {
-      return cancelled;
-    }
-
-    protected AbortProcedureResponse abortProcedureResult(
-        final AbortProcedureRequest request) throws IOException {
-      return admin.executeCallable(new MasterCallable<AbortProcedureResponse>(
-          admin.getConnection(), admin.getRpcControllerFactory()) {
-        @Override
-        protected AbortProcedureResponse rpcCall() throws Exception {
-          return master.abortProcedure(getRpcController(), request);
-        }
-      });
-    }
-
-    @Override
-    public V get() throws InterruptedException, ExecutionException {
-      // TODO: should we ever spin forever?
-      // fix HBASE-21715. TODO: If the function call get() without timeout limit is not allowed,
-      // is it possible to compose instead of inheriting from the class Future for this class?
-      try {
-        return get(admin.getProcedureTimeout, TimeUnit.MILLISECONDS);
-      } catch (TimeoutException e) {
-        LOG.warn("Failed to get the procedure with procId=" + procId + " throws exception " + e
-            .getMessage(), e);
-        return null;
-      }
-    }
-
-    @Override
-    public V get(long timeout, TimeUnit unit)
-        throws InterruptedException, ExecutionException, TimeoutException {
-      if (!done) {
-        long deadlineTs = EnvironmentEdgeManager.currentTime() + unit.toMillis(timeout);
-        try {
-          try {
-            // if the master support procedures, try to wait the result
-            if (procId != null) {
-              result = waitProcedureResult(procId, deadlineTs);
-            }
-            // if we don't have a proc result, try the compatibility wait
-            if (!procResultFound) {
-              result = waitOperationResult(deadlineTs);
-            }
-            result = postOperationResult(result, deadlineTs);
-            done = true;
-          } catch (IOException e) {
-            result = postOperationFailure(e, deadlineTs);
-            done = true;
-          }
-        } catch (IOException e) {
-          exception = new ExecutionException(e);
-          done = true;
-        }
-      }
-      if (exception != null) {
-        throw exception;
-      }
-      return result;
-    }
-
-    @Override
-    public boolean isDone() {
-      return done;
-    }
-
-    protected HBaseAdmin getAdmin() {
-      return admin;
-    }
-
-    private V waitProcedureResult(long procId, long deadlineTs)
-        throws IOException, TimeoutException, InterruptedException {
-      GetProcedureResultRequest request = GetProcedureResultRequest.newBuilder()
-          .setProcId(procId)
-          .build();
-
-      int tries = 0;
-      IOException serviceEx = null;
-      while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
-        GetProcedureResultResponse response = null;
-        try {
-          // Try to fetch the result
-          response = getProcedureResult(request);
-        } catch (IOException e) {
-          serviceEx = unwrapException(e);
-
-          // the master may be down
-          LOG.warn("failed to get the procedure result procId=" + procId, serviceEx);
-
-          // Not much to do, if we have a DoNotRetryIOException
-          if (serviceEx instanceof DoNotRetryIOException) {
-            // TODO: looks like there is no way to unwrap this exception and get the proper
-            // UnsupportedOperationException aside from looking at the message.
-            // anyway, if we fail here we just failover to the compatibility side
-            // and that is always a valid solution.
-            LOG.warn("Proc-v2 is unsupported on this master: " + serviceEx.getMessage(), serviceEx);
-            procResultFound = false;
-            return null;
-          }
-        }
-
-        // If the procedure is no longer running, we should have a result
-        if (response != null && response.getState() != GetProcedureResultResponse.State.RUNNING) {
-          procResultFound = response.getState() != GetProcedureResultResponse.State.NOT_FOUND;
-          return convertResult(response);
-        }
-
-        try {
-          Thread.sleep(getAdmin().getPauseTime(tries++));
-        } catch (InterruptedException e) {
-          throw new InterruptedException(
-            "Interrupted while waiting for the result of proc " + procId);
-        }
-      }
-      if (serviceEx != null) {
-        throw serviceEx;
-      } else {
-        throw new TimeoutException("The procedure " + procId + " is still running");
-      }
-    }
-
-    private static IOException unwrapException(IOException e) {
-      if (e instanceof RemoteException) {
-        return ((RemoteException)e).unwrapRemoteException();
-      }
-      return e;
-    }
-
-    protected GetProcedureResultResponse getProcedureResult(final GetProcedureResultRequest request)
-        throws IOException {
-      return admin.executeCallable(new MasterCallable<GetProcedureResultResponse>(
-          admin.getConnection(), admin.getRpcControllerFactory()) {
-        @Override
-        protected GetProcedureResultResponse rpcCall() throws Exception {
-          return master.getProcedureResult(getRpcController(), request);
-        }
-      });
-    }
-
-    /**
-     * Convert the procedure result response to a specified type.
-     * @param response the procedure result object to parse
-     * @return the result data of the procedure.
-     */
-    protected V convertResult(final GetProcedureResultResponse response) throws IOException {
-      if (response.hasException()) {
-        throw ForeignExceptionUtil.toIOException(response.getException());
-      }
-      return null;
-    }
-
-    /**
-     * Fallback implementation in case the procedure is not supported by the server.
-     * It should try to wait until the operation is completed.
-     * @param deadlineTs the timestamp after which this method should throw a TimeoutException
-     * @return the result data of the operation
-     */
-    protected V waitOperationResult(final long deadlineTs)
-        throws IOException, TimeoutException {
-      return null;
-    }
-
-    /**
-     * Called after the operation is completed and the result fetched. this allows to perform extra
-     * steps after the procedure is completed. it allows to apply transformations to the result that
-     * will be returned by get().
-     * @param result the result of the procedure
-     * @param deadlineTs the timestamp after which this method should throw a TimeoutException
-     * @return the result of the procedure, which may be the same as the passed one
-     */
-    protected V postOperationResult(final V result, final long deadlineTs)
-        throws IOException, TimeoutException {
-      return result;
-    }
-
-    /**
-     * Called after the operation is terminated with a failure.
-     * this allows to perform extra steps after the procedure is terminated.
-     * it allows to apply transformations to the result that will be returned by get().
-     * The default implementation will rethrow the exception
-     * @param exception the exception got from fetching the result
-     * @param deadlineTs the timestamp after which this method should throw a TimeoutException
-     * @return the result of the procedure, which may be the same as the passed one
-     */
-    protected V postOperationFailure(final IOException exception, final long deadlineTs)
-        throws IOException, TimeoutException {
-      throw exception;
-    }
-
-    protected interface WaitForStateCallable {
-      boolean checkState(int tries) throws IOException;
-      void throwInterruptedException() throws InterruptedIOException;
-      void throwTimeoutException(long elapsed) throws TimeoutException;
-    }
-
-    protected void waitForState(final long deadlineTs, final WaitForStateCallable callable)
-        throws IOException, TimeoutException {
-      int tries = 0;
-      IOException serverEx = null;
-      long startTime = EnvironmentEdgeManager.currentTime();
-      while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
-        serverEx = null;
-        try {
-          if (callable.checkState(tries)) {
-            return;
-          }
-        } catch (IOException e) {
-          serverEx = e;
-        }
-        try {
-          Thread.sleep(getAdmin().getPauseTime(tries++));
-        } catch (InterruptedException e) {
-          callable.throwInterruptedException();
-        }
-      }
-      if (serverEx != null) {
-        throw unwrapException(serverEx);
-      } else {
-        callable.throwTimeoutException(EnvironmentEdgeManager.currentTime() - startTime);
-      }
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Evolving
-  protected static abstract class TableFuture<V> extends ProcedureFuture<V> {
-    private final TableName tableName;
-
-    public TableFuture(final HBaseAdmin admin, final TableName tableName, final Long procId) {
-      super(admin, procId);
-      this.tableName = tableName;
-    }
-
-    @Override
-    public String toString() {
-      return getDescription();
-    }
-
-    /**
-     * @return the table name
-     */
-    protected TableName getTableName() {
-      return tableName;
-    }
-
-    /**
-     * @return the table descriptor
-     */
-    protected TableDescriptor getDescriptor() throws IOException {
-      return getAdmin().getDescriptor(getTableName());
-    }
-
-    /**
-     * @return the operation type like CREATE, DELETE, DISABLE etc.
-     */
-    public abstract String getOperationType();
-
-    /**
-     * @return a description of the operation
-     */
-    protected String getDescription() {
-      return "Operation: " + getOperationType() + ", " + "Table Name: " +
-        tableName.getNameWithNamespaceInclAsString() + ", procId: " + procId;
-    }
-
-    protected abstract class TableWaitForStateCallable implements WaitForStateCallable {
-      @Override
-      public void throwInterruptedException() throws InterruptedIOException {
-        throw new InterruptedIOException("Interrupted while waiting for " + getDescription());
-      }
-
-      @Override
-      public void throwTimeoutException(long elapsedTime) throws TimeoutException {
-        throw new TimeoutException(
-          getDescription() + " has not completed after " + elapsedTime + "ms");
-      }
-    }
-
-    @Override
-    protected V postOperationResult(final V result, final long deadlineTs)
-        throws IOException, TimeoutException {
-      LOG.info(getDescription() + " completed");
-      return super.postOperationResult(result, deadlineTs);
-    }
-
-    @Override
-    protected V postOperationFailure(final IOException exception, final long deadlineTs)
-        throws IOException, TimeoutException {
-      LOG.info(getDescription() + " failed with " + exception.getMessage());
-      return super.postOperationFailure(exception, deadlineTs);
-    }
-
-    protected void waitForTableEnabled(final long deadlineTs)
-        throws IOException, TimeoutException {
-      waitForState(deadlineTs, new TableWaitForStateCallable() {
-        @Override
-        public boolean checkState(int tries) throws IOException {
-          try {
-            if (getAdmin().isTableAvailable(tableName)) {
-              return true;
-            }
-          } catch (TableNotFoundException tnfe) {
-            LOG.debug("Table " + tableName.getNameWithNamespaceInclAsString()
-                + " was not enabled, sleeping. tries=" + tries);
-          }
-          return false;
-        }
-      });
-    }
-
-    protected void waitForTableDisabled(final long deadlineTs)
-        throws IOException, TimeoutException {
-      waitForState(deadlineTs, new TableWaitForStateCallable() {
-        @Override
-        public boolean checkState(int tries) throws IOException {
-          return getAdmin().isTableDisabled(tableName);
-        }
-      });
-    }
-
-    protected void waitTableNotFound(final long deadlineTs)
-        throws IOException, TimeoutException {
-      waitForState(deadlineTs, new TableWaitForStateCallable() {
-        @Override
-        public boolean checkState(int tries) throws IOException {
-          return !getAdmin().tableExists(tableName);
-        }
-      });
-    }
-
-    protected void waitForAllRegionsOnline(final long deadlineTs, final byte[][] splitKeys)
-        throws IOException, TimeoutException {
-      final TableDescriptor desc = getDescriptor();
-      final AtomicInteger actualRegCount = new AtomicInteger(0);
-      final MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
-        @Override
-        public boolean visit(Result rowResult) throws IOException {
-          RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
-          if (list == null) {
-            LOG.warn("No serialized HRegionInfo in " + rowResult);
-            return true;
-          }
-          HRegionLocation l = list.getRegionLocation();
-          if (l == null) {
-            return true;
-          }
-          if (!l.getRegion().getTable().equals(desc.getTableName())) {
-            return false;
-          }
-          if (l.getRegion().isOffline() || l.getRegion().isSplit()) {
-            return true;
-          }
-          HRegionLocation[] locations = list.getRegionLocations();
-          for (HRegionLocation location : locations) {
-            if (location == null) continue;
-            ServerName serverName = location.getServerName();
-            // Make sure that regions are assigned to server
-            if (serverName != null && serverName.getAddress() != null) {
-              actualRegCount.incrementAndGet();
-            }
-          }
-          return true;
-        }
-      };
-
-      int tries = 0;
-      int numRegs = (splitKeys == null ? 1 : splitKeys.length + 1) * desc.getRegionReplication();
-      while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
-        actualRegCount.set(0);
-        MetaTableAccessor.scanMetaForTableRegions(getAdmin().getConnection(), visitor,
-          desc.getTableName());
-        if (actualRegCount.get() == numRegs) {
-          // all the regions are online
-          return;
-        }
-
-        try {
-          Thread.sleep(getAdmin().getPauseTime(tries++));
-        } catch (InterruptedException e) {
-          throw new InterruptedIOException("Interrupted when opening" + " regions; "
-              + actualRegCount.get() + " of " + numRegs + " regions processed so far");
-        }
-      }
-      throw new TimeoutException("Only " + actualRegCount.get() + " of " + numRegs
-          + " regions are online; retries exhausted.");
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Evolving
-  protected static abstract class NamespaceFuture extends ProcedureFuture<Void> {
-    private final String namespaceName;
-
-    public NamespaceFuture(final HBaseAdmin admin, final String namespaceName, final Long procId) {
-      super(admin, procId);
-      this.namespaceName = namespaceName;
-    }
-
-    /**
-     * @return the namespace name
-     */
-    protected String getNamespaceName() {
-      return namespaceName;
-    }
-
-    /**
-     * @return the operation type like CREATE_NAMESPACE, DELETE_NAMESPACE, etc.
-     */
-    public abstract String getOperationType();
-
-    @Override
-    public String toString() {
-      return "Operation: " + getOperationType() + ", Namespace: " + getNamespaceName();
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Evolving
-  private static class ReplicationFuture extends ProcedureFuture<Void> {
-    private final String peerId;
-    private final Supplier<String> getOperation;
-
-    public ReplicationFuture(HBaseAdmin admin, String peerId, Long procId,
-        Supplier<String> getOperation) {
-      super(admin, procId);
-      this.peerId = peerId;
-      this.getOperation = getOperation;
-    }
-
-    @Override
-    public String toString() {
-      return "Operation: " + getOperation.get() + ", peerId: " + peerId;
-    }
-  }
-
-  @Override
-  public List<SecurityCapability> getSecurityCapabilities() throws IOException {
-    try {
-      return executeCallable(new MasterCallable<List<SecurityCapability>>(getConnection(),
-          getRpcControllerFactory()) {
-        @Override
-        protected List<SecurityCapability> rpcCall() throws Exception {
-          SecurityCapabilitiesRequest req = SecurityCapabilitiesRequest.newBuilder().build();
-          return ProtobufUtil.toSecurityCapabilityList(
-            master.getSecurityCapabilities(getRpcController(), req).getCapabilitiesList());
-        }
-      });
-    } catch (IOException e) {
-      if (e instanceof RemoteException) {
-        e = ((RemoteException)e).unwrapRemoteException();
-      }
-      throw e;
-    }
-  }
-
-  @Override
-  public boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException {
-    return splitOrMergeSwitch(enabled, synchronous, MasterSwitchType.SPLIT);
-  }
-
-  @Override
-  public boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException {
-    return splitOrMergeSwitch(enabled, synchronous, MasterSwitchType.MERGE);
-  }
-
-  private boolean splitOrMergeSwitch(boolean enabled, boolean synchronous,
-      MasterSwitchType switchType) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Boolean rpcCall() throws Exception {
-        MasterProtos.SetSplitOrMergeEnabledResponse response = master.setSplitOrMergeEnabled(
-          getRpcController(),
-          RequestConverter.buildSetSplitOrMergeEnabledRequest(enabled, synchronous, switchType));
-        return response.getPrevValueList().get(0);
-      }
-    });
-  }
-
-  @Override
-  public boolean isSplitEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Boolean rpcCall() throws Exception {
-        return master.isSplitOrMergeEnabled(getRpcController(),
-          RequestConverter.buildIsSplitOrMergeEnabledRequest(MasterSwitchType.SPLIT)).getEnabled();
-      }
-    });
-  }
-
-  @Override
-  public boolean isMergeEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Boolean rpcCall() throws Exception {
-        return master.isSplitOrMergeEnabled(getRpcController(),
-          RequestConverter.buildIsSplitOrMergeEnabledRequest(MasterSwitchType.MERGE)).getEnabled();
-      }
-    });
-  }
-
-  private RpcControllerFactory getRpcControllerFactory() {
-    return this.rpcControllerFactory;
-  }
-
-  @Override
-  public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
-      boolean enabled) throws IOException {
-    AddReplicationPeerResponse response = executeCallable(
-      new MasterCallable<AddReplicationPeerResponse>(getConnection(), getRpcControllerFactory()) {
-        @Override
-        protected AddReplicationPeerResponse rpcCall() throws Exception {
-          return master.addReplicationPeer(getRpcController(),
-            RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled));
-        }
-      });
-    return new ReplicationFuture(this, peerId, response.getProcId(), () -> "ADD_REPLICATION_PEER");
-  }
-
-  @Override
-  public Future<Void> removeReplicationPeerAsync(String peerId) throws IOException {
-    RemoveReplicationPeerResponse response =
-      executeCallable(new MasterCallable<RemoveReplicationPeerResponse>(getConnection(),
-          getRpcControllerFactory()) {
-        @Override
-        protected RemoveReplicationPeerResponse rpcCall() throws Exception {
-          return master.removeReplicationPeer(getRpcController(),
-            RequestConverter.buildRemoveReplicationPeerRequest(peerId));
-        }
-      });
-    return new ReplicationFuture(this, peerId, response.getProcId(),
-      () -> "REMOVE_REPLICATION_PEER");
-  }
-
-  @Override
-  public Future<Void> enableReplicationPeerAsync(final String peerId) throws IOException {
-    EnableReplicationPeerResponse response =
-      executeCallable(new MasterCallable<EnableReplicationPeerResponse>(getConnection(),
-          getRpcControllerFactory()) {
-        @Override
-        protected EnableReplicationPeerResponse rpcCall() throws Exception {
-          return master.enableReplicationPeer(getRpcController(),
-            RequestConverter.buildEnableReplicationPeerRequest(peerId));
-        }
-      });
-    return new ReplicationFuture(this, peerId, response.getProcId(),
-      () -> "ENABLE_REPLICATION_PEER");
-  }
-
-  @Override
-  public Future<Void> disableReplicationPeerAsync(final String peerId) throws IOException {
-    DisableReplicationPeerResponse response =
-      executeCallable(new MasterCallable<DisableReplicationPeerResponse>(getConnection(),
-          getRpcControllerFactory()) {
-        @Override
-        protected DisableReplicationPeerResponse rpcCall() throws Exception {
... 16266 lines suppressed ...