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/05/16 14:54:16 UTC

[hbase] branch HBASE-21512 updated (2602d32 -> d72a8e2)

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

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


 discard 2602d32  HBASE-22351 Increase the wait time when creating table for TestProcedurePriority
 discard a934577  HBASE-22328 NPE in RegionReplicaReplicationEndpoint
 discard 0fd487b  HBASE-22036 Rewrite TestScannerHeartbeatMessages
 discard d1e8112  HBASE-22239 Also catch RemoteException in SyncReplicationTestBase.verifyReplicationRequestRejection
 discard 85cdc94  HBASE-22302 Fix TestHbck
 discard 9ca7be4  HBASE-22297 Fix TestRegionMergeTransitionOnCluster and TestSplitTransactionOnCluster
 discard ccc6c2b  HBASE-22295 Fix TestClientOperationTimeout
 discard 37560dd  HBASE-22281 Fix failed shell UTs
 discard 4886b2a  HBASE-22223 Implement RegionLocator based on AsyncTableRegionLocator
 discard 4883b00  HBASE-22238 Fix TestRpcControllerFactory
 discard 8289fc8  HBASE-21725 Implement BufferedMutator Based on AsyncBufferedMutator
 discard 4d68ae7  HBASE-22241 Fix TestRegionServerCoprocessorEndpoint
 discard 1dbcc44  HBASE-21718 Implement Admin based on AsyncAdmin
     new 18ea03c  HBASE-21718 Implement Admin based on AsyncAdmin
     new f405b9d  HBASE-22241 Fix TestRegionServerCoprocessorEndpoint
     new 82f0dbf  HBASE-21725 Implement BufferedMutator Based on AsyncBufferedMutator
     new c02174c  HBASE-22238 Fix TestRpcControllerFactory
     new 9dcedb7  HBASE-22223 Implement RegionLocator based on AsyncTableRegionLocator
     new e5fe6ae  HBASE-22281 Fix failed shell UTs
     new 564f7db  HBASE-22295 Fix TestClientOperationTimeout
     new 48ebece  HBASE-22297 Fix TestRegionMergeTransitionOnCluster and TestSplitTransactionOnCluster
     new 002fbcf  HBASE-22302 Fix TestHbck
     new 4770fbc  HBASE-22239 Also catch RemoteException in SyncReplicationTestBase.verifyReplicationRequestRejection
     new abea8b2  HBASE-22036 Rewrite TestScannerHeartbeatMessages
     new 3e5cd7d  HBASE-22328 NPE in RegionReplicaReplicationEndpoint
     new d72a8e2  HBASE-22351 Increase the wait time when creating table for TestProcedurePriority

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

 * -- * -- B -- O -- O -- O   (2602d32)
            \
             N -- N -- N   refs/heads/HBASE-21512 (d72a8e2)

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

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

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


Summary of changes:
 .../org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java   | 11 +++++------
 1 file changed, 5 insertions(+), 6 deletions(-)


[hbase] 11/13: HBASE-22036 Rewrite TestScannerHeartbeatMessages

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

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

commit abea8b25bb0264499ebe00cf5514144604e3aa9c
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Thu Apr 25 18:18:58 2019 +0800

    HBASE-22036 Rewrite TestScannerHeartbeatMessages
---
 .../hbase/client/ScanPerNextResultScanner.java     | 147 +++++++++++++++++++++
 .../regionserver/TestScannerHeartbeatMessages.java |  71 +++++-----
 2 files changed, 187 insertions(+), 31 deletions(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/ScanPerNextResultScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/ScanPerNextResultScanner.java
new file mode 100644
index 0000000..c8665e9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/ScanPerNextResultScanner.java
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayDeque;
+import java.util.Queue;
+import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+
+/**
+ * A ResultScanner which will only send request to RS when there are no cached results when calling
+ * next, just like the ResultScanner in the old time. Mainly used for writing UTs, that we can
+ * control when to send request to RS. The default ResultScanner implementation will fetch in
+ * background.
+ */
+@InterfaceAudience.Private
+public class ScanPerNextResultScanner implements ResultScanner, AdvancedScanResultConsumer {
+
+  private final AsyncTable<AdvancedScanResultConsumer> table;
+
+  private final Scan scan;
+
+  private final Queue<Result> queue = new ArrayDeque<>();
+
+  private ScanMetrics scanMetrics;
+
+  private boolean closed = false;
+
+  private Throwable error;
+
+  private ScanResumer resumer;
+
+  public ScanPerNextResultScanner(AsyncTable<AdvancedScanResultConsumer> table, Scan scan) {
+    this.table = table;
+    this.scan = scan;
+  }
+
+  @Override
+  public synchronized void onError(Throwable error) {
+    this.error = error;
+    notifyAll();
+  }
+
+  @Override
+  public synchronized void onComplete() {
+    closed = true;
+    notifyAll();
+  }
+
+  @Override
+  public void onScanMetricsCreated(ScanMetrics scanMetrics) {
+    this.scanMetrics = scanMetrics;
+  }
+
+  @Override
+  public synchronized void onNext(Result[] results, ScanController controller) {
+    assert results.length > 0;
+    if (closed) {
+      controller.terminate();
+      return;
+    }
+    for (Result result : results) {
+      queue.add(result);
+    }
+    notifyAll();
+    resumer = controller.suspend();
+  }
+
+  @Override
+  public synchronized void onHeartbeat(ScanController controller) {
+    if (closed) {
+      controller.terminate();
+      return;
+    }
+    if (scan.isNeedCursorResult()) {
+      controller.cursor().ifPresent(c -> queue.add(Result.createCursorResult(c)));
+    }
+  }
+
+  @Override
+  public synchronized Result next() throws IOException {
+    if (queue.isEmpty()) {
+      if (resumer != null) {
+        resumer.resume();
+        resumer = null;
+      } else {
+        table.scan(scan, this);
+      }
+    }
+    while (queue.isEmpty()) {
+      if (closed) {
+        return null;
+      }
+      if (error != null) {
+        Throwables.propagateIfPossible(error, IOException.class);
+        throw new IOException(error);
+      }
+      try {
+        wait();
+      } catch (InterruptedException e) {
+        throw new InterruptedIOException();
+      }
+    }
+    return queue.poll();
+  }
+
+  @Override
+  public synchronized void close() {
+    closed = true;
+    queue.clear();
+    if (resumer != null) {
+      resumer.resume();
+      resumer = null;
+    }
+    notifyAll();
+  }
+
+  @Override
+  public boolean renewLease() {
+    // The renew lease operation will be handled in background
+    return false;
+  }
+
+  @Override
+  public ScanMetrics getScanMetrics() {
+    return scanMetrics;
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
index ea9f7e7..7a21941 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
@@ -39,11 +39,16 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTestConst;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanPerNextResultScanner;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -58,10 +63,10 @@ import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 
@@ -75,11 +80,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRespon
  * the client when the server has exceeded the time limit during the processing of the scan. When
  * the time limit is reached, the server will return to the Client whatever Results it has
  * accumulated (potentially empty).
- * <p/>
- * TODO: with async client based sync client, we will fetch result in background which makes this
- * test broken. We need to find another way to implement the test.
  */
-@Ignore
 @Category(MediumTests.class)
 public class TestScannerHeartbeatMessages {
 
@@ -89,7 +90,7 @@ public class TestScannerHeartbeatMessages {
 
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
-  private static Table TABLE = null;
+  private static AsyncConnection CONN;
 
   /**
    * Table configuration
@@ -141,16 +142,19 @@ public class TestScannerHeartbeatMessages {
     conf.setLong(StoreScanner.HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK, 1);
     TEST_UTIL.startMiniCluster(1);
 
-    TABLE = createTestTable(TABLE_NAME, ROWS, FAMILIES, QUALIFIERS, VALUE);
+    createTestTable(TABLE_NAME, ROWS, FAMILIES, QUALIFIERS, VALUE);
+
+    Configuration newConf = new Configuration(conf);
+    newConf.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, CLIENT_TIMEOUT);
+    newConf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, CLIENT_TIMEOUT);
+    CONN = ConnectionFactory.createAsyncConnection(newConf).get();
   }
 
-  static Table createTestTable(TableName name, byte[][] rows, byte[][] families,
-      byte[][] qualifiers, byte[] cellValue) throws IOException {
+  static void createTestTable(TableName name, byte[][] rows, byte[][] families, byte[][] qualifiers,
+      byte[] cellValue) throws IOException {
     Table ht = TEST_UTIL.createTable(name, families);
     List<Put> puts = createPuts(rows, families, qualifiers, cellValue);
     ht.put(puts);
-    ht.getConfiguration().setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, CLIENT_TIMEOUT);
-    return ht;
   }
 
   /**
@@ -177,6 +181,7 @@ public class TestScannerHeartbeatMessages {
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
+    Closeables.close(CONN, true);
     TEST_UTIL.shutdownMiniCluster();
   }
 
@@ -311,26 +316,28 @@ public class TestScannerHeartbeatMessages {
         scan.setMaxResultSize(Long.MAX_VALUE);
         scan.setCaching(Integer.MAX_VALUE);
         scan.setFilter(new SparseCellFilter());
-        ResultScanner scanner = TABLE.getScanner(scan);
-        int num = 0;
-        while (scanner.next() != null) {
-          num++;
+        try (ScanPerNextResultScanner scanner =
+          new ScanPerNextResultScanner(CONN.getTable(TABLE_NAME), scan)) {
+          int num = 0;
+          while (scanner.next() != null) {
+            num++;
+          }
+          assertEquals(1, num);
         }
-        assertEquals(1, num);
-        scanner.close();
 
         scan = new Scan();
         scan.setMaxResultSize(Long.MAX_VALUE);
         scan.setCaching(Integer.MAX_VALUE);
         scan.setFilter(new SparseCellFilter());
         scan.setAllowPartialResults(true);
-        scanner = TABLE.getScanner(scan);
-        num = 0;
-        while (scanner.next() != null) {
-          num++;
+        try (ScanPerNextResultScanner scanner =
+          new ScanPerNextResultScanner(CONN.getTable(TABLE_NAME), scan)) {
+          int num = 0;
+          while (scanner.next() != null) {
+            num++;
+          }
+          assertEquals(NUM_FAMILIES * NUM_QUALIFIERS, num);
         }
-        assertEquals(NUM_FAMILIES * NUM_QUALIFIERS, num);
-        scanner.close();
 
         return null;
       }
@@ -349,13 +356,14 @@ public class TestScannerHeartbeatMessages {
         scan.setMaxResultSize(Long.MAX_VALUE);
         scan.setCaching(Integer.MAX_VALUE);
         scan.setFilter(new SparseRowFilter());
-        ResultScanner scanner = TABLE.getScanner(scan);
-        int num = 0;
-        while (scanner.next() != null) {
-          num++;
+        try (ScanPerNextResultScanner scanner =
+          new ScanPerNextResultScanner(CONN.getTable(TABLE_NAME), scan)) {
+          int num = 0;
+          while (scanner.next() != null) {
+            num++;
+          }
+          assertEquals(1, num);
         }
-        assertEquals(1, num);
-        scanner.close();
 
         return null;
       }
@@ -374,8 +382,9 @@ public class TestScannerHeartbeatMessages {
   private void testEquivalenceOfScanWithHeartbeats(final Scan scan, int rowSleepTime,
       int cfSleepTime, boolean sleepBeforeCf) throws Exception {
     disableSleeping();
-    final ResultScanner scanner = TABLE.getScanner(scan);
-    final ResultScanner scannerWithHeartbeats = TABLE.getScanner(scan);
+    AsyncTable<AdvancedScanResultConsumer> table = CONN.getTable(TABLE_NAME);
+    final ResultScanner scanner = new ScanPerNextResultScanner(table, scan);
+    final ResultScanner scannerWithHeartbeats = new ScanPerNextResultScanner(table, scan);
 
     Result r1 = null;
     Result r2 = null;


[hbase] 10/13: HBASE-22239 Also catch RemoteException in SyncReplicationTestBase.verifyReplicationRequestRejection

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

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

commit 4770fbc2de8bc8970083b33047bfc55e9385f954
Author: zhangduo <zh...@apache.org>
AuthorDate: Thu Apr 25 22:43:51 2019 +0800

    HBASE-22239 Also catch RemoteException in SyncReplicationTestBase.verifyReplicationRequestRejection
---
 .../hadoop/hbase/replication/SyncReplicationTestBase.java  | 14 ++++++++++++--
 1 file changed, 12 insertions(+), 2 deletions(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
index e0d112d..fd8df32 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -51,6 +53,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.apache.hadoop.ipc.RemoteException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
@@ -247,6 +250,12 @@ public class SyncReplicationTestBase {
     }
   }
 
+  private void assertRejection(Throwable error) {
+    assertThat(error, instanceOf(DoNotRetryIOException.class));
+    assertTrue(error.getMessage().contains("Reject to apply to sink cluster"));
+    assertTrue(error.getMessage().contains(TABLE_NAME.toString()));
+  }
+
   protected final void verifyReplicationRequestRejection(HBaseTestingUtility utility,
       boolean expectedRejection) throws Exception {
     HRegionServer regionServer = utility.getRSForFirstRegionInTable(TABLE_NAME);
@@ -264,9 +273,10 @@ public class SyncReplicationTestBase {
         ReplicationProtbufUtil.replicateWALEntry(
           connection.getRegionServerAdmin(regionServer.getServerName()), entries, null, null, null);
         fail("Should throw IOException when sync-replication state is in A or DA");
+      } catch (RemoteException e) {
+        assertRejection(e.unwrapRemoteException());
       } catch (DoNotRetryIOException e) {
-        assertTrue(e.getMessage().contains("Reject to apply to sink cluster"));
-        assertTrue(e.getMessage().contains(TABLE_NAME.toString()));
+        assertRejection(e);
       }
     }
   }


[hbase] 09/13: HBASE-22302 Fix TestHbck

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

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

commit 002fbcf36a586131dec1752f21cf5afe3f02703d
Author: zhangduo <zh...@apache.org>
AuthorDate: Wed Apr 24 22:30:02 2019 +0800

    HBASE-22302 Fix TestHbck
---
 .../src/test/java/org/apache/hadoop/hbase/client/TestHbck.java        | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
index d9a7ca9..ee277d9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
@@ -29,7 +29,6 @@ import java.util.Optional;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -273,7 +272,8 @@ public class TestHbck {
     FailingSplitAfterMetaUpdatedMasterObserver observer = master.getMasterCoprocessorHost()
         .findCoprocessor(FailingSplitAfterMetaUpdatedMasterObserver.class);
     assertNotNull(observer);
-    try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
+    try {
+      AsyncAdmin admin = TEST_UTIL.getAsyncConnection().getAdmin();
       byte[] splitKey = Bytes.toBytes("bcd");
       admin.split(TableName.valueOf(testTable), splitKey);
       observer.latch.await(5000, TimeUnit.MILLISECONDS);


[hbase] 12/13: HBASE-22328 NPE in RegionReplicaReplicationEndpoint

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

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

commit 3e5cd7d43b2086c5983968936d4530cc0d091f75
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Tue Apr 30 16:33:58 2019 +0800

    HBASE-22328 NPE in RegionReplicaReplicationEndpoint
---
 .../regionserver/RegionReplicaReplicationEndpoint.java     | 14 ++++++++------
 1 file changed, 8 insertions(+), 6 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
index 65cf9a8..cc2650f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
@@ -151,21 +151,23 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
   private void getRegionLocations(CompletableFuture<RegionLocations> future,
       TableDescriptor tableDesc, byte[] encodedRegionName, byte[] row, boolean reload) {
     FutureUtils.addListener(connection.getRegionLocations(tableDesc.getTableName(), row, reload),
-      (r, e) -> {
+      (locs, e) -> {
         if (e != null) {
           future.completeExceptionally(e);
           return;
         }
         // if we are not loading from cache, just return
         if (reload) {
-          future.complete(r);
+          future.complete(locs);
           return;
         }
         // check if the number of region replicas is correct, and also the primary region name
-        // matches
-        if (r.size() == tableDesc.getRegionReplication() && Bytes.equals(
-          r.getDefaultRegionLocation().getRegion().getEncodedNameAsBytes(), encodedRegionName)) {
-          future.complete(r);
+        // matches, and also there is no null elements in the returned RegionLocations
+        if (locs.size() == tableDesc.getRegionReplication() &&
+          locs.size() == locs.numNonNullElements() &&
+          Bytes.equals(locs.getDefaultRegionLocation().getRegion().getEncodedNameAsBytes(),
+            encodedRegionName)) {
+          future.complete(locs);
         } else {
           // reload again as the information in cache maybe stale
           getRegionLocations(future, tableDesc, encodedRegionName, row, true);


[hbase] 05/13: HBASE-22223 Implement RegionLocator based on AsyncTableRegionLocator

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

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

commit 9dcedb7b2c4b7099d0ff01316a142cf64672cb8e
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Tue Apr 16 16:52:54 2019 +0800

    HBASE-22223 Implement RegionLocator based on AsyncTableRegionLocator
---
 .../hadoop/hbase/client/AsyncConnectionImpl.java   | 12 +---
 .../client/ConnectionOverAsyncConnection.java      | 16 ++---
 .../RegionLocatorOverAsyncTableRegionLocator.java  | 70 ++++++++++++++++++++++
 3 files changed, 76 insertions(+), 22 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 4a00412..84e1da6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -28,7 +28,6 @@ import static org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLE
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import java.io.IOException;
-import java.io.UncheckedIOException;
 import java.net.SocketAddress;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -208,7 +207,7 @@ class AsyncConnectionImpl implements AsyncConnection {
     metrics.ifPresent(MetricsConnection::shutdown);
     ConnectionOverAsyncConnection c = this.conn;
     if (c != null) {
-      c.closeConnImpl();
+      c.closePool();
     }
     closed = true;
   }
@@ -362,14 +361,7 @@ class AsyncConnectionImpl implements AsyncConnection {
       if (c != null) {
         return c;
       }
-      try {
-        c = new ConnectionOverAsyncConnection(this,
-          ConnectionFactory.createConnectionImpl(conf, null, user));
-      } catch (IOException e) {
-        // TODO: finally we will not rely on ConnectionImplementation anymore and there will no
-        // IOException here.
-        throw new UncheckedIOException(e);
-      }
+      c = new ConnectionOverAsyncConnection(this);
       this.conn = c;
     }
     return c;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java
index 8ec7ab8..861aab0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java
@@ -43,20 +43,12 @@ class ConnectionOverAsyncConnection implements Connection {
 
   private volatile ExecutorService batchPool = null;
 
-  protected final AsyncConnectionImpl conn;
-
-  /**
-   * @deprecated we can not implement all the related stuffs at once so keep it here for now, will
-   *             remove it after we implement all the stuffs, like Admin, RegionLocator, etc.
-   */
-  @Deprecated
-  private final ConnectionImplementation oldConn;
+  private final AsyncConnectionImpl conn;
 
   private final ConnectionConfiguration connConf;
 
-  ConnectionOverAsyncConnection(AsyncConnectionImpl conn, ConnectionImplementation oldConn) {
+  ConnectionOverAsyncConnection(AsyncConnectionImpl conn) {
     this.conn = conn;
-    this.oldConn = oldConn;
     this.connConf = new ConnectionConfiguration(conn.getConfiguration());
   }
 
@@ -109,7 +101,7 @@ class ConnectionOverAsyncConnection implements Connection {
 
   @Override
   public RegionLocator getRegionLocator(TableName tableName) throws IOException {
-    return oldConn.getRegionLocator(tableName);
+    return new RegionLocatorOverAsyncTableRegionLocator(conn.getRegionLocator(tableName));
   }
 
   @Override
@@ -129,7 +121,7 @@ class ConnectionOverAsyncConnection implements Connection {
 
   // will be called from AsyncConnection, to avoid infinite loop as in the above method we will call
   // AsyncConnection.close.
-  void closeConnImpl() {
+  void closePool() {
     ExecutorService batchPool = this.batchPool;
     if (batchPool != null) {
       ConnectionUtils.shutdownPool(batchPool);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLocatorOverAsyncTableRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLocatorOverAsyncTableRegionLocator.java
new file mode 100644
index 0000000..5e21e3b
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLocatorOverAsyncTableRegionLocator.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.apache.hadoop.hbase.util.FutureUtils.get;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The {@link RegionLocator} implementation based on {@link AsyncTableRegionLocator}.
+ */
+@InterfaceAudience.Private
+class RegionLocatorOverAsyncTableRegionLocator implements RegionLocator {
+
+  private final AsyncTableRegionLocator locator;
+
+  RegionLocatorOverAsyncTableRegionLocator(AsyncTableRegionLocator locator) {
+    this.locator = locator;
+  }
+
+  @Override
+  public void close() {
+  }
+
+  @Override
+  public HRegionLocation getRegionLocation(byte[] row, int replicaId, boolean reload)
+      throws IOException {
+    return get(locator.getRegionLocation(row, replicaId, reload));
+  }
+
+  @Override
+  public List<HRegionLocation> getRegionLocations(byte[] row, boolean reload) throws IOException {
+    return get(locator.getRegionLocations(row, reload));
+  }
+
+  @Override
+  public void clearRegionLocationCache() {
+    locator.clearRegionLocationCache();
+  }
+
+  @Override
+  public List<HRegionLocation> getAllRegionLocations() throws IOException {
+    return get(locator.getAllRegionLocations());
+  }
+
+  @Override
+  public TableName getName() {
+    return locator.getName();
+  }
+
+}


[hbase] 07/13: HBASE-22295 Fix TestClientOperationTimeout

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

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

commit 564f7dbec08df3587e02494ef81cbf965092544f
Author: zhangduo <zh...@apache.org>
AuthorDate: Tue Apr 23 21:54:31 2019 +0800

    HBASE-22295 Fix TestClientOperationTimeout
    
    Signed-off-by: Michael Stack <st...@apache.org>
---
 .../test/java/org/apache/hadoop/hbase/TestClientOperationTimeout.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientOperationTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientOperationTimeout.java
index 2ce34a9..52f0c7d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientOperationTimeout.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientOperationTimeout.java
@@ -120,7 +120,7 @@ public class TestClientOperationTimeout {
    * Tests that a get on a table throws {@link SocketTimeoutException} when the operation takes
    * longer than 'hbase.client.operation.timeout'.
    */
-  @Test(expected = SocketTimeoutException.class)
+  @Test(expected = RetriesExhaustedException.class)
   public void testGetTimeout() throws Exception {
     DELAY_GET = 600;
     TABLE.get(new Get(ROW));
@@ -130,7 +130,7 @@ public class TestClientOperationTimeout {
    * Tests that a put on a table throws {@link SocketTimeoutException} when the operation takes
    * longer than 'hbase.client.operation.timeout'.
    */
-  @Test(expected = SocketTimeoutException.class)
+  @Test(expected = RetriesExhaustedException.class)
   public void testPutTimeout() throws Exception {
     DELAY_MUTATE = 600;
 


[hbase] 13/13: HBASE-22351 Increase the wait time when creating table for TestProcedurePriority

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

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

commit d72a8e2d6fc81833068db242631b5b7ece7c9a0a
Author: zhangduo <zh...@apache.org>
AuthorDate: Thu May 2 21:09:26 2019 +0800

    HBASE-22351 Increase the wait time when creating table for TestProcedurePriority
---
 .../apache/hadoop/hbase/master/procedure/TestProcedurePriority.java   | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java
index 1cfe17b..36f31e0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java
@@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
@@ -107,6 +108,7 @@ public class TestProcedurePriority {
     UTIL.getConfiguration().setLong(ProcedureExecutor.WORKER_KEEP_ALIVE_TIME_CONF_KEY, 5000);
     UTIL.getConfiguration().setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 4);
     UTIL.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, MyCP.class.getName());
+    UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 100);
     UTIL.startMiniCluster(3);
     CORE_POOL_SIZE =
       UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor().getCorePoolSize();
@@ -118,7 +120,7 @@ public class TestProcedurePriority {
           .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build()));
     }
     for (Future<?> future : futures) {
-      future.get(1, TimeUnit.MINUTES);
+      future.get(3, TimeUnit.MINUTES);
     }
     UTIL.getAdmin().balance(true);
     UTIL.waitUntilNoRegionsInTransition();


[hbase] 04/13: HBASE-22238 Fix TestRpcControllerFactory

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

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

commit c02174cbb9d99878387ecd4cec6eb4e6c0db97be
Author: zhangduo <zh...@apache.org>
AuthorDate: Sun Apr 14 21:30:34 2019 +0800

    HBASE-22238 Fix TestRpcControllerFactory
---
 .../hbase/client/TestRpcControllerFactory.java     | 171 ++++++++++-----------
 1 file changed, 77 insertions(+), 94 deletions(-)

diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
index 2d60733..bdda4e8 100644
--- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
+++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Scan.ReadType;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.ProtobufCoprocessorService;
 import org.apache.hadoop.hbase.ipc.DelegatingHBaseRpcController;
@@ -52,12 +53,12 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ConcurrentHashMulti
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Multiset;
 
-@Category({MediumTests.class, ClientTests.class})
+@Category({ MediumTests.class, ClientTests.class })
 public class TestRpcControllerFactory {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestRpcControllerFactory.class);
+    HBaseClassTestRule.forClass(TestRpcControllerFactory.class);
 
   public static class StaticRpcControllerFactory extends RpcControllerFactory {
 
@@ -85,7 +86,6 @@ public class TestRpcControllerFactory {
 
     private static Multiset<Integer> GROUPED_PRIORITY = ConcurrentHashMultiset.create();
     private static AtomicInteger INT_PRIORITY = new AtomicInteger();
-    private static AtomicInteger TABLE_PRIORITY = new AtomicInteger();
 
     public CountingRpcController(HBaseRpcController delegate) {
       super(delegate);
@@ -93,24 +93,8 @@ public class TestRpcControllerFactory {
 
     @Override
     public void setPriority(int priority) {
-      int oldPriority = getPriority();
-      super.setPriority(priority);
-      int newPriority = getPriority();
-      if (newPriority != oldPriority) {
-        INT_PRIORITY.incrementAndGet();
-        GROUPED_PRIORITY.add(priority);
-      }
-    }
-
-    @Override
-    public void setPriority(TableName tn) {
-      super.setPriority(tn);
-      // ignore counts for system tables - it could change and we really only want to check on what
-      // the client should change
-      if (tn != null && !tn.isSystemTable()) {
-        TABLE_PRIORITY.incrementAndGet();
-      }
-
+      INT_PRIORITY.incrementAndGet();
+      GROUPED_PRIORITY.add(priority);
     }
   }
 
@@ -120,7 +104,7 @@ public class TestRpcControllerFactory {
   public TestName name = new TestName();
 
   @BeforeClass
-  public static void setup() throws Exception {
+  public static void setUp() throws Exception {
     // load an endpoint so we have an endpoint to test - it doesn't matter which one, but
     // this is already in tests, so we can just use it.
     Configuration conf = UTIL.getConfiguration();
@@ -131,7 +115,7 @@ public class TestRpcControllerFactory {
   }
 
   @AfterClass
-  public static void teardown() throws Exception {
+  public static void tearDown() throws Exception {
     UTIL.shutdownMiniCluster();
   }
 
@@ -154,84 +138,83 @@ public class TestRpcControllerFactory {
     // change one of the connection properties so we get a new Connection with our configuration
     conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT + 1);
 
-    Connection connection = ConnectionFactory.createConnection(conf);
-    Table table = connection.getTable(tableName);
-    byte[] row = Bytes.toBytes("row");
-    Put p = new Put(row);
-    p.addColumn(fam1, fam1, Bytes.toBytes("val0"));
-    table.put(p);
-
-    Integer counter = 1;
-    counter = verifyCount(counter);
-
-    Delete d = new Delete(row);
-    d.addColumn(fam1, fam1);
-    table.delete(d);
-    counter = verifyCount(counter);
-
-    Put p2 = new Put(row);
-    p2.addColumn(fam1, Bytes.toBytes("qual"), Bytes.toBytes("val1"));
-    table.batch(Lists.newArrayList(p, p2), null);
-    // this only goes to a single server, so we don't need to change the count here
-    counter = verifyCount(counter);
-
-    Append append = new Append(row);
-    append.addColumn(fam1, fam1, Bytes.toBytes("val2"));
-    table.append(append);
-    counter = verifyCount(counter);
-
-    // and check the major lookup calls as well
-    Get g = new Get(row);
-    table.get(g);
-    counter = verifyCount(counter);
-
-    ResultScanner scan = table.getScanner(fam1);
-    scan.next();
-    scan.close();
-    counter = verifyCount(counter + 1);
-
-    Get g2 = new Get(row);
-    table.get(Lists.newArrayList(g, g2));
-    // same server, so same as above for not changing count
-    counter = verifyCount(counter);
-
-    // make sure all the scanner types are covered
-    Scan scanInfo = new Scan(row);
-    // regular small
-    scanInfo.setSmall(true);
-    counter = doScan(table, scanInfo, counter);
-
-    // reversed, small
-    scanInfo.setReversed(true);
-    counter = doScan(table, scanInfo, counter);
-
-    // reversed, regular
-    scanInfo.setSmall(false);
-    counter = doScan(table, scanInfo, counter + 1);
-
-    // make sure we have no priority count
-    verifyPriorityGroupCount(HConstants.ADMIN_QOS, 0);
-    // lets set a custom priority on a get
-    Get get = new Get(row);
-    get.setPriority(HConstants.ADMIN_QOS);
-    table.get(get);
-    verifyPriorityGroupCount(HConstants.ADMIN_QOS, 1);
-
-    table.close();
-    connection.close();
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+        Table table = connection.getTable(tableName)) {
+      byte[] row = Bytes.toBytes("row");
+      Put p = new Put(row);
+      p.addColumn(fam1, fam1, Bytes.toBytes("val0"));
+      table.put(p);
+
+      Integer counter = 1;
+      counter = verifyCount(counter);
+
+      Delete d = new Delete(row);
+      d.addColumn(fam1, fam1);
+      table.delete(d);
+      counter = verifyCount(counter);
+
+      Put p2 = new Put(row);
+      p2.addColumn(fam1, Bytes.toBytes("qual"), Bytes.toBytes("val1"));
+      table.batch(Lists.newArrayList(p, p2), null);
+      // this only goes to a single server, so we don't need to change the count here
+      counter = verifyCount(counter);
+
+      Append append = new Append(row);
+      append.addColumn(fam1, fam1, Bytes.toBytes("val2"));
+      table.append(append);
+      counter = verifyCount(counter);
+
+      // and check the major lookup calls as well
+      Get g = new Get(row);
+      table.get(g);
+      counter = verifyCount(counter);
+
+      ResultScanner scan = table.getScanner(fam1);
+      scan.next();
+      scan.close();
+      counter = verifyCount(counter + 1);
+
+      Get g2 = new Get(row);
+      table.get(Lists.newArrayList(g, g2));
+      // same server, so same as above for not changing count
+      counter = verifyCount(counter);
+
+      // make sure all the scanner types are covered
+      Scan scanInfo = new Scan().withStartRow(row);
+      // regular small
+      scanInfo.setReadType(ReadType.PREAD);
+      counter = doScan(table, scanInfo, counter);
+
+      // reversed, small
+      scanInfo.setReversed(true);
+      counter = doScan(table, scanInfo, counter);
+
+      // reversed, regular
+      scanInfo.setReadType(ReadType.STREAM);
+      counter = doScan(table, scanInfo, counter + 1);
+
+      // make sure we have no priority count
+      verifyPriorityGroupCount(HConstants.ADMIN_QOS, 0);
+      // lets set a custom priority on a get
+      Get get = new Get(row);
+      get.setPriority(HConstants.ADMIN_QOS);
+      table.get(get);
+      // we will reset the controller for setting the call timeout so it will lead to an extra
+      // setPriority
+      verifyPriorityGroupCount(HConstants.ADMIN_QOS, 2);
+    }
   }
 
   int doScan(Table table, Scan scan, int expectedCount) throws IOException {
-    ResultScanner results = table.getScanner(scan);
-    results.next();
-    results.close();
+    try (ResultScanner results = table.getScanner(scan)) {
+      results.next();
+    }
     return verifyCount(expectedCount);
   }
 
   int verifyCount(Integer counter) {
-    assertTrue(CountingRpcController.TABLE_PRIORITY.get() >= counter);
-    assertEquals(0, CountingRpcController.INT_PRIORITY.get());
-    return CountingRpcController.TABLE_PRIORITY.get() + 1;
+    assertTrue(CountingRpcController.INT_PRIORITY.get() >= counter);
+    return CountingRpcController.GROUPED_PRIORITY.count(HConstants.NORMAL_QOS) + 1;
   }
 
   void verifyPriorityGroupCount(int priorityLevel, int count) {


[hbase] 01/13: HBASE-21718 Implement Admin based on AsyncAdmin

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

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

commit 18ea03cf994c2a9d15fecc15abc2f8ff1bc78fda
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Fri Apr 12 15:08:11 2019 +0800

    HBASE-21718 Implement Admin based on AsyncAdmin
---
 .../hadoop/hbase/backup/util/RestoreTool.java      |   2 +-
 .../apache/hadoop/hbase/backup/TestBackupBase.java |   6 +-
 .../hbase/backup/TestBackupDeleteRestore.java      |   4 +-
 .../hadoop/hbase/backup/TestBackupMerge.java       |   4 +-
 .../hbase/backup/TestBackupMultipleDeletes.java    |   5 +-
 .../hadoop/hbase/backup/TestBackupSystemTable.java |   2 +-
 .../hadoop/hbase/backup/TestFullBackupSet.java     |   4 +-
 .../hbase/backup/TestFullBackupSetRestoreSet.java  |   6 +-
 .../hadoop/hbase/backup/TestFullRestore.java       |  16 +-
 .../hadoop/hbase/backup/TestIncrementalBackup.java |   7 +-
 .../backup/TestIncrementalBackupDeleteTable.java   |   7 +-
 .../TestIncrementalBackupMergeWithFailures.java    |   4 +-
 .../backup/TestIncrementalBackupWithBulkLoad.java  |   6 +-
 .../backup/TestIncrementalBackupWithFailures.java  |   5 +-
 .../hadoop/hbase/backup/TestRemoteBackup.java      |   4 +-
 .../hadoop/hbase/backup/TestRemoteRestore.java     |   4 +-
 .../hbase/backup/TestRestoreBoundaryTests.java     |   6 +-
 .../hbase/backup/TestSystemTableSnapshot.java      |   4 +-
 .../hadoop/hbase/AsyncMetaTableAccessor.java       |  29 +-
 .../java/org/apache/hadoop/hbase/client/Admin.java |  50 +-
 .../hadoop/hbase/client/AdminOverAsyncAdmin.java   | 945 +++++++++++++++++++++
 .../client/ConnectionOverAsyncConnection.java      |   2 +-
 .../hadoop/hbase/client/ConnectionUtils.java       |  18 +
 .../client/CoprocessorBlockingRpcCallback.java     |  68 ++
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java |   9 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |  37 +-
 .../client/RegionCoprocessorRpcChannelImpl.java    |  21 +-
 .../hbase/client/SyncCoprocessorRpcChannel.java    |   3 +
 .../hadoop/hbase/client/TableOverAsyncTable.java   |  51 +-
 .../hadoop/hbase/client/TestInterfaceAlign.java    |   2 +
 .../apache/hadoop/hbase/PerformanceEvaluation.java |   6 +-
 .../hadoop/hbase/rest/client/TestRemoteTable.java  |   4 +-
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |  42 +-
 .../org/apache/hadoop/hbase/client/TestAdmin1.java |  40 +-
 .../org/apache/hadoop/hbase/client/TestAdmin2.java |  35 +-
 .../hadoop/hbase/client/TestFromClientSide.java    |   7 +-
 .../hadoop/hbase/client/TestFromClientSide3.java   |  10 +-
 .../client/TestSnapshotDFSTemporaryDirectory.java  |   5 +-
 .../client/TestSnapshotTemporaryDirectory.java     |  12 +-
 .../hbase/client/TestSplitOrMergeStatus.java       |  13 +-
 .../hbase/coprocessor/TestMasterObserver.java      |   2 +-
 .../org/apache/hadoop/hbase/master/TestMaster.java |   4 +-
 .../hbase/master/TestMasterMetricsWrapper.java     |   4 +-
 .../master/TestMergeTableRegionsWhileRSCrash.java  |   2 +-
 .../hbase/master/TestSplitRegionWhileRSCrash.java  |   2 +-
 .../master/assignment/TestAssignmentOnRSCrash.java |   2 +-
 .../TestMasterAbortWhileMergingTable.java          |   2 +-
 .../assignment/TestModifyTableWhileMerging.java    |   2 +-
 .../TestCleanupCompactedFileOnRegionClose.java     |   6 +-
 .../regionserver/TestEndToEndSplitTransaction.java |   9 +-
 .../TestNewVersionBehaviorFromClientSide.java      |   2 +-
 .../hbase/regionserver/TestRegionServerAbort.java  |   2 +-
 .../replication/regionserver/TestReplicator.java   |   4 +-
 .../hbase/snapshot/SnapshotTestingUtils.java       |  29 -
 .../snapshot/TestFlushSnapshotFromClient.java      |  41 +-
 .../hadoop/hbase/tool/TestBulkLoadHFiles.java      |   6 +-
 .../hbase/thrift2/ThriftHBaseServiceHandler.java   |   6 +-
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   |   9 +-
 58 files changed, 1283 insertions(+), 356 deletions(-)

diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
index 92254fa..e03bfe4 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
@@ -486,7 +486,7 @@ public class RestoreTool {
         LOG.info("Creating target table '" + targetTableName + "'");
         byte[][] keys;
         if (regionDirList == null || regionDirList.size() == 0) {
-          admin.createTable(htd, null);
+          admin.createTable(htd);
         } else {
           keys = generateBoundaryKeys(regionDirList);
           // create table using table descriptor and region boundaries
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
index e0fca20..64978bc 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
@@ -26,7 +26,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
@@ -53,7 +52,6 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
@@ -342,7 +340,7 @@ public class TestBackupBase {
   @AfterClass
   public static void tearDown() throws Exception {
     try{
-      SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
+      SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getAdmin());
     } catch (Exception e) {
     }
     SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
@@ -416,7 +414,7 @@ public class TestBackupBase {
   protected static void createTables() throws Exception {
     long tid = System.currentTimeMillis();
     table1 = TableName.valueOf("test-" + tid);
-    HBaseAdmin ha = TEST_UTIL.getHBaseAdmin();
+    Admin ha = TEST_UTIL.getAdmin();
 
     // Create namespaces
     NamespaceDescriptor desc1 = NamespaceDescriptor.create("ns1").build();
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
index 74176e3..f649b92 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
@@ -24,8 +24,8 @@ import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -61,7 +61,7 @@ public class TestBackupDeleteRestore extends TestBackupBase {
     assertTrue(checkSucceeded(backupId));
     LOG.info("backup complete");
     int numRows = TEST_UTIL.countRows(table1);
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    Admin hba = TEST_UTIL.getAdmin();
     // delete row
     try (Table table = TEST_UTIL.getConnection().getTable(table1)) {
       Delete delete = new Delete(Bytes.toBytes("row0"));
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMerge.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMerge.java
index beacef3..1a8638c 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMerge.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMerge.java
@@ -24,9 +24,9 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.Assert;
@@ -62,7 +62,7 @@ public class TestBackupMerge extends TestBackupBase {
 
     Connection conn = ConnectionFactory.createConnection(conf1);
 
-    HBaseAdmin admin = (HBaseAdmin) conn.getAdmin();
+    Admin admin = conn.getAdmin();
     BackupAdminImpl client = new BackupAdminImpl(conn);
 
     BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
index bffa480..538488b 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
@@ -26,9 +26,9 @@ import java.util.Set;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -59,9 +59,8 @@ public class TestBackupMultipleDeletes extends TestBackupBase {
     // #1 - create full backup for all tables
     LOG.info("create full backup image for all tables");
     List<TableName> tables = Lists.newArrayList(table1, table2);
-    HBaseAdmin admin = null;
     Connection conn = ConnectionFactory.createConnection(conf1);
-    admin = (HBaseAdmin) conn.getAdmin();
+    Admin admin = conn.getAdmin();
     BackupAdmin client = new BackupAdminImpl(conn);
     BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
     String backupIdFull = client.backupTables(request);
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
index aa6e5dd..5d48fc5 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
@@ -119,7 +119,7 @@ public class TestBackupSystemTable {
   }
 
   private void cleanBackupTable() throws IOException {
-    Admin admin = UTIL.getHBaseAdmin();
+    Admin admin = UTIL.getAdmin();
     admin.disableTable(BackupSystemTable.getTableName(conf));
     admin.truncateTable(BackupSystemTable.getTableName(conf), true);
     if (admin.isTableDisabled(BackupSystemTable.getTableName(conf))) {
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
index 89ff571..7a3aec4 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.ClassRule;
@@ -80,7 +80,7 @@ public class TestFullBackupSet extends TestBackupBase {
       // Run backup
       ret = ToolRunner.run(conf1, new RestoreDriver(), args);
       assertTrue(ret == 0);
-      HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+      Admin hba = TEST_UTIL.getAdmin();
       assertTrue(hba.tableExists(table1_restore));
       // Verify number of rows in both tables
       assertEquals(TEST_UTIL.countRows(table1), TEST_UTIL.countRows(table1_restore));
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java
index ca70f6a..3543133 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.ClassRule;
@@ -76,7 +76,7 @@ public class TestFullBackupSetRestoreSet extends TestBackupBase {
       // Run backup
       ret = ToolRunner.run(conf1, new RestoreDriver(), args);
       assertTrue(ret == 0);
-      HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+      Admin hba = TEST_UTIL.getAdmin();
       assertTrue(hba.tableExists(table1_restore));
       // Verify number of rows in both tables
       assertEquals(TEST_UTIL.countRows(table1), TEST_UTIL.countRows(table1_restore));
@@ -118,7 +118,7 @@ public class TestFullBackupSetRestoreSet extends TestBackupBase {
       // Run backup
       ret = ToolRunner.run(conf1, new RestoreDriver(), args);
       assertTrue(ret == 0);
-      HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+      Admin hba = TEST_UTIL.getAdmin();
       assertTrue(hba.tableExists(table1));
       // Verify number of rows in both tables
       assertEquals(count, TEST_UTIL.countRows(table1));
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
index 2201e2f..f5ad0d7 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
@@ -26,7 +26,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.ClassRule;
@@ -66,7 +66,7 @@ public class TestFullRestore extends TestBackupBase {
     BackupAdmin client = getBackupAdmin();
     client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,
       tableset, tablemap, false));
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    Admin hba = TEST_UTIL.getAdmin();
     assertTrue(hba.tableExists(table1_restore));
     TEST_UTIL.deleteTable(table1_restore);
     hba.close();
@@ -88,7 +88,7 @@ public class TestFullRestore extends TestBackupBase {
     int ret = ToolRunner.run(conf1, new RestoreDriver(), args);
 
     assertTrue(ret == 0);
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    Admin hba = TEST_UTIL.getAdmin();
     assertTrue(hba.tableExists(table1_restore));
     TEST_UTIL.deleteTable(table1_restore);
     hba.close();
@@ -110,7 +110,7 @@ public class TestFullRestore extends TestBackupBase {
     int ret = ToolRunner.run(conf1, new RestoreDriver(), args);
     assertTrue(ret == 0);
     //Verify that table has not been restored
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    Admin hba = TEST_UTIL.getAdmin();
     assertFalse(hba.tableExists(table1_restore));
   }
 
@@ -131,7 +131,7 @@ public class TestFullRestore extends TestBackupBase {
     BackupAdmin client = getBackupAdmin();
     client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,
       restore_tableset, tablemap, false));
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    Admin hba = TEST_UTIL.getAdmin();
     assertTrue(hba.tableExists(table2_restore));
     assertTrue(hba.tableExists(table3_restore));
     TEST_UTIL.deleteTable(table2_restore);
@@ -162,7 +162,7 @@ public class TestFullRestore extends TestBackupBase {
     int ret = ToolRunner.run(conf1, new RestoreDriver(), args);
 
     assertTrue(ret == 0);
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    Admin hba = TEST_UTIL.getAdmin();
     assertTrue(hba.tableExists(table2_restore));
     assertTrue(hba.tableExists(table3_restore));
     TEST_UTIL.deleteTable(table2_restore);
@@ -210,7 +210,7 @@ public class TestFullRestore extends TestBackupBase {
     int ret = ToolRunner.run(conf1, new RestoreDriver(), args);
     assertTrue(ret == 0);
 
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    Admin hba = TEST_UTIL.getAdmin();
     assertTrue(hba.tableExists(table1));
     hba.close();
   }
@@ -256,7 +256,7 @@ public class TestFullRestore extends TestBackupBase {
     int ret = ToolRunner.run(conf1, new RestoreDriver(), args);
 
     assertTrue(ret == 0);
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    Admin hba = TEST_UTIL.getAdmin();
     assertTrue(hba.tableExists(table2));
     assertTrue(hba.tableExists(table3));
     hba.close();
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
index 35a77ea..d7c2cd0 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
@@ -29,9 +29,9 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -93,8 +93,7 @@ public class TestIncrementalBackup extends TestBackupBase {
       int NB_ROWS_FAM3 = 6;
       insertIntoTable(conn, table1, fam3Name, 3, NB_ROWS_FAM3).close();
       insertIntoTable(conn, table1, mobName, 3, NB_ROWS_FAM3).close();
-      HBaseAdmin admin = null;
-      admin = (HBaseAdmin) conn.getAdmin();
+      Admin admin = conn.getAdmin();
       BackupAdminImpl client = new BackupAdminImpl(conn);
       BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
       String backupIdFull = client.backupTables(request);
@@ -182,7 +181,7 @@ public class TestIncrementalBackup extends TestBackupBase {
                 tablesRestoreFull, tablesMapFull, true));
 
       // #6.1 - check tables for full restore
-      HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
+      Admin hAdmin = TEST_UTIL.getAdmin();
       assertTrue(hAdmin.tableExists(table1_restore));
       assertTrue(hAdmin.tableExists(table2_restore));
       hAdmin.close();
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
index 08834f2..837de4d 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
@@ -24,9 +24,9 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -64,9 +64,8 @@ public class TestIncrementalBackupDeleteTable extends TestBackupBase {
     LOG.info("create full backup image for all tables");
 
     List<TableName> tables = Lists.newArrayList(table1, table2);
-    HBaseAdmin admin = null;
     Connection conn = ConnectionFactory.createConnection(conf1);
-    admin = (HBaseAdmin) conn.getAdmin();
+    Admin admin = conn.getAdmin();
     BackupAdminImpl client = new BackupAdminImpl(conn);
 
     BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
@@ -105,7 +104,7 @@ public class TestIncrementalBackupDeleteTable extends TestBackupBase {
       tablesRestoreFull, tablesMapFull, false));
 
     // #5.1 - check tables for full restore
-    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
+    Admin hAdmin = TEST_UTIL.getAdmin();
     assertTrue(hAdmin.tableExists(table1_restore));
     assertTrue(hAdmin.tableExists(table2_restore));
 
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
index 7351258..1bde63b 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
@@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
 import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
 import org.apache.hadoop.hbase.backup.mapreduce.MapReduceHFileSplitterJob;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Pair;
@@ -235,7 +235,7 @@ public class TestIncrementalBackupMergeWithFailures extends TestBackupBase {
 
     Connection conn = ConnectionFactory.createConnection(conf1);
 
-    HBaseAdmin admin = (HBaseAdmin) conn.getAdmin();
+    Admin admin = conn.getAdmin();
     BackupAdminImpl client = new BackupAdminImpl(conn);
 
     BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
index 4b02077..60aa635 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
@@ -26,9 +26,9 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -70,7 +70,7 @@ public class TestIncrementalBackupWithBulkLoad extends TestBackupBase {
 
     List<TableName> tables = Lists.newArrayList(table1);
     Connection conn = ConnectionFactory.createConnection(conf1);
-    HBaseAdmin admin = (HBaseAdmin) conn.getAdmin();
+    Admin admin = conn.getAdmin();
     BackupAdminImpl client = new BackupAdminImpl(conn);
 
     BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
@@ -119,7 +119,7 @@ public class TestIncrementalBackupWithBulkLoad extends TestBackupBase {
     // Delete all data in table1
     TEST_UTIL.deleteTableData(table1);
     // #5.1 - check tables for full restore */
-    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
+    Admin hAdmin = TEST_UTIL.getAdmin();
 
     // #6 - restore incremental backup for table1
     TableName[] tablesRestoreIncMultiple = new TableName[] { table1 };
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
index f6725d9..546cf41 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
@@ -32,9 +32,9 @@ import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
 import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
 import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -90,8 +90,7 @@ public class TestIncrementalBackupWithFailures extends TestBackupBase {
     int NB_ROWS_FAM3 = 6;
     insertIntoTable(conn, table1, fam3Name, 3, NB_ROWS_FAM3).close();
 
-    HBaseAdmin admin = null;
-    admin = (HBaseAdmin) conn.getAdmin();
+    Admin admin = conn.getAdmin();
     BackupAdminImpl client = new BackupAdminImpl(conn);
 
     BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
index 05826e2..2d99e0d 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
@@ -26,9 +26,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
@@ -126,7 +126,7 @@ public class TestRemoteBackup extends TestBackupBase {
       tablesRestoreFull, tablesMapFull, false));
 
     // check tables for full restore
-    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
+    Admin hAdmin = TEST_UTIL.getAdmin();
     assertTrue(hAdmin.tableExists(table1_restore));
 
     // #5.2 - checking row count of tables for full restore
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
index 25ebca2..d670144 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
@@ -22,7 +22,7 @@ import static org.junit.Assert.assertTrue;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -61,7 +61,7 @@ public class TestRemoteRestore extends TestBackupBase {
     getBackupAdmin().restore(
       BackupUtils.createRestoreRequest(BACKUP_REMOTE_ROOT_DIR, backupId, false, tableset,
         tablemap, false));
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    Admin hba = TEST_UTIL.getAdmin();
     assertTrue(hba.tableExists(table1_restore));
     TEST_UTIL.deleteTable(table1_restore);
     hba.close();
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
index 07f57cc..a6808cd 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
@@ -23,7 +23,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -55,7 +55,7 @@ public class TestRestoreBoundaryTests extends TestBackupBase {
     getBackupAdmin().restore(
       BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tableset, tablemap,
         false));
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    Admin hba = TEST_UTIL.getAdmin();
     assertTrue(hba.tableExists(table1_restore));
     TEST_UTIL.deleteTable(table1_restore);
   }
@@ -76,7 +76,7 @@ public class TestRestoreBoundaryTests extends TestBackupBase {
     getBackupAdmin().restore(
       BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, restore_tableset,
         tablemap, false));
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    Admin hba = TEST_UTIL.getAdmin();
     assertTrue(hba.tableExists(table2_restore));
     assertTrue(hba.tableExists(table3_restore));
     TEST_UTIL.deleteTable(table2_restore);
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java
index b93fa77..bd29512 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.backup;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
@@ -47,7 +47,7 @@ public class TestSystemTableSnapshot extends TestBackupBase {
 
     TableName backupSystem = BackupSystemTable.getTableName(conf1);
 
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    Admin hba = TEST_UTIL.getAdmin();
     String snapshotName = "sysTable";
     hba.snapshot(snapshotName, backupSystem);
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
index 4a886d1..d04ea52 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.client.Scan.ReadType;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -80,23 +79,17 @@ public class AsyncMetaTableAccessor {
       TableName tableName) {
     CompletableFuture<Optional<TableState>> future = new CompletableFuture<>();
     Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getStateColumn());
-    long time = EnvironmentEdgeManager.currentTime();
-    try {
-      get.setTimeRange(0, time);
-      addListener(metaTable.get(get), (result, error) -> {
-        if (error != null) {
-          future.completeExceptionally(error);
-          return;
-        }
-        try {
-          future.complete(getTableState(result));
-        } catch (IOException e) {
-          future.completeExceptionally(e);
-        }
-      });
-    } catch (IOException ioe) {
-      future.completeExceptionally(ioe);
-    }
+    addListener(metaTable.get(get), (result, error) -> {
+      if (error != null) {
+        future.completeExceptionally(error);
+        return;
+      }
+      try {
+        future.complete(getTableState(result));
+      } catch (IOException e) {
+        future.completeExceptionally(e);
+      }
+    });
     return future;
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 14abb6e..707f5e7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -60,7 +60,6 @@ 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.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -255,13 +254,14 @@ public interface Admin extends Abortable, Closeable {
   Future<Void> createTableAsync(TableDescriptor desc) throws IOException;
 
   /**
-   * Creates a new table but does not block and wait for it to come online. You can use
-   * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
-   * ExecutionException if there was an error while executing the operation or TimeoutException in
-   * case the wait timeout was not long enough to allow the operation to complete.
-   * <p/>
-   * Throws IllegalArgumentException Bad table name, if the split keys are repeated and if the split
-   * key has empty byte array.
+   * Creates a new table but does not block and wait for it to come online.
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
+   * It may throw ExecutionException if there was an error while executing the operation
+   * or TimeoutException in case the wait timeout was not long enough to allow the
+   * operation to complete.
+   * Throws IllegalArgumentException Bad table name, if the split keys
+   *    are repeated and if the split key has empty byte array.
+   *
    * @param desc table descriptor for table
    * @param splitKeys keys to check if the table has been created with all split keys
    * @throws IOException if a remote or network exception occurs
@@ -699,29 +699,7 @@ public interface Admin extends Abortable, Closeable {
   void move(byte[] encodedRegionName) throws IOException;
 
   /**
-   * Move the region <code>rencodedRegionName</code> to <code>destServerName</code>.
-   * @param encodedRegionName The encoded region name; i.e. the hash that makes up the region name
-   *          suffix: e.g. if regionname is
-   *          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
-   *          then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>.
-   * @param destServerName The servername of the destination regionserver. If passed the empty byte
-   *          array we'll assign to a random server. A server name is made of host, port and
-   *          startcode. Here is an example: <code> host187.example.com,60020,1289493121758</code>
-   * @throws IOException if we can't find a region named <code>encodedRegionName</code>
-   * @deprecated Use {@link #move(byte[], ServerName)} instead. And if you want to move the region
-   *             to a random server, please use {@link #move(byte[])}.
-   */
-  @Deprecated
-  default void move(byte[] encodedRegionName, byte[] destServerName) throws IOException {
-    if (destServerName == null || destServerName.length == 0) {
-      move(encodedRegionName);
-    } else {
-      move(encodedRegionName, ServerName.valueOf(Bytes.toString(destServerName)));
-    }
-  }
-
-  /**
-   * Move the region <code>rencodedRegionName</code> to <code>destServerName</code>.
+   * Move the region <code>encodedRegionName</code> to <code>destServerName</code>.
    * @param encodedRegionName The encoded region name; i.e. the hash that makes up the region name
    *          suffix: e.g. if regionname is
    *          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
@@ -1063,9 +1041,7 @@ public interface Admin extends Abortable, Closeable {
    * @return a {@link RegionMetrics} list of all regions hosted on a region server
    * @throws IOException if a remote or network exception occurs
    */
-  default List<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException {
-    return getRegionMetrics(serverName, null);
-  }
+  List<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException;
 
   /**
    * Get {@link RegionMetrics} of all regions hosted on a regionserver for a table.
@@ -1654,7 +1630,10 @@ public interface Admin extends Abortable, Closeable {
    * </pre></blockquote></div>
    *
    * @return A MasterCoprocessorRpcChannel instance
+   * @deprecated since 3.0.0, will removed in 4.0.0. This is too low level, please stop using it any
+   *             more. Use the coprocessorService methods in {@link AsyncAdmin} instead.
    */
+  @Deprecated
   CoprocessorRpcChannel coprocessorService();
 
 
@@ -1679,7 +1658,10 @@ public interface Admin extends Abortable, Closeable {
    *
    * @param serverName the server name to which the endpoint call is made
    * @return A RegionServerCoprocessorRpcChannel instance
+   * @deprecated since 3.0.0, will removed in 4.0.0. This is too low level, please stop using it any
+   *             more. Use the coprocessorService methods in {@link AsyncAdmin} instead.
    */
+  @Deprecated
   CoprocessorRpcChannel coprocessorService(ServerName serverName);
 
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
new file mode 100644
index 0000000..599e5d6
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
@@ -0,0 +1,945 @@
+/**
+ * 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.setCoprocessorError;
+import static org.apache.hadoop.hbase.util.FutureUtils.get;
+
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.Message;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcChannel;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Future;
+import java.util.regex.Pattern;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CacheEvictionStats;
+import org.apache.hadoop.hbase.ClusterMetrics;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.replication.TableCFs;
+import org.apache.hadoop.hbase.client.security.SecurityCapability;
+import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.hadoop.hbase.quotas.QuotaFilter;
+import org.apache.hadoop.hbase.quotas.QuotaSettings;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
+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.access.GetUserPermissionsRequest;
+import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.access.UserPermission;
+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.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The {@link Admin} implementation which is based on an {@link AsyncAdmin}.
+ */
+@InterfaceAudience.Private
+class AdminOverAsyncAdmin implements Admin {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AdminOverAsyncAdmin.class);
+
+  private volatile boolean aborted = false;
+
+  private final Connection conn;
+
+  private final RawAsyncHBaseAdmin admin;
+
+  private final int operationTimeout;
+
+  private final int syncWaitTimeout;
+
+  public AdminOverAsyncAdmin(Connection conn, RawAsyncHBaseAdmin admin) {
+    this.conn = conn;
+    this.admin = admin;
+    this.operationTimeout = conn.getConfiguration().getInt(
+      HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
+    this.syncWaitTimeout =
+      conn.getConfiguration().getInt("hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
+  }
+
+  @Override
+  public int getOperationTimeout() {
+    return operationTimeout;
+  }
+
+  @Override
+  public int getSyncWaitTimeout() {
+    return syncWaitTimeout;
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+    LOG.warn("Aborting becasue of {}", why, e);
+    this.aborted = true;
+  }
+
+  @Override
+  public boolean isAborted() {
+    return aborted;
+  }
+
+  @Override
+  public Connection getConnection() {
+    return conn;
+  }
+
+  @Override
+  public boolean tableExists(TableName tableName) throws IOException {
+    return get(admin.tableExists(tableName));
+  }
+
+  @Override
+  public List<TableDescriptor> listTableDescriptors() throws IOException {
+    return get(admin.listTableDescriptors());
+  }
+
+  @Override
+  public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
+      throws IOException {
+    return get(admin.listTableDescriptors(pattern, includeSysTables));
+  }
+
+  @Override
+  public TableName[] listTableNames() throws IOException {
+    return get(admin.listTableNames()).toArray(new TableName[0]);
+  }
+
+  @Override
+  public TableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException {
+    return get(admin.listTableNames(pattern, includeSysTables)).toArray(new TableName[0]);
+  }
+
+  @Override
+  public TableDescriptor getDescriptor(TableName tableName)
+      throws TableNotFoundException, IOException {
+    return get(admin.getDescriptor(tableName));
+  }
+
+  @Override
+  public void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
+      throws IOException {
+    get(admin.createTable(desc, startKey, endKey, numRegions));
+  }
+
+  @Override
+  public Future<Void> createTableAsync(TableDescriptor desc) throws IOException {
+    return admin.createTable(desc);
+  }
+
+  @Override
+  public Future<Void> createTableAsync(TableDescriptor desc, byte[][] splitKeys)
+      throws IOException {
+    return admin.createTable(desc, splitKeys);
+  }
+
+  @Override
+  public Future<Void> deleteTableAsync(TableName tableName) throws IOException {
+    return admin.deleteTable(tableName);
+  }
+
+  @Override
+  public Future<Void> truncateTableAsync(TableName tableName, boolean preserveSplits)
+      throws IOException {
+    return admin.truncateTable(tableName, preserveSplits);
+  }
+
+  @Override
+  public Future<Void> enableTableAsync(TableName tableName) throws IOException {
+    return admin.enableTable(tableName);
+  }
+
+  @Override
+  public Future<Void> disableTableAsync(TableName tableName) throws IOException {
+    return admin.disableTable(tableName);
+  }
+
+  @Override
+  public boolean isTableEnabled(TableName tableName) throws IOException {
+    return get(admin.isTableEnabled(tableName));
+  }
+
+  @Override
+  public boolean isTableDisabled(TableName tableName) throws IOException {
+    return get(admin.isTableDisabled(tableName));
+  }
+
+  @Override
+  public boolean isTableAvailable(TableName tableName) throws IOException {
+    return get(admin.isTableAvailable(tableName));
+  }
+
+  @Override
+  public Future<Void> addColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily)
+      throws IOException {
+    return admin.addColumnFamily(tableName, columnFamily);
+  }
+
+  @Override
+  public Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily)
+      throws IOException {
+    return admin.deleteColumnFamily(tableName, columnFamily);
+  }
+
+  @Override
+  public Future<Void> modifyColumnFamilyAsync(TableName tableName,
+      ColumnFamilyDescriptor columnFamily) throws IOException {
+    return admin.modifyColumnFamily(tableName, columnFamily);
+  }
+
+  @Override
+  public List<RegionInfo> getRegions(ServerName serverName) throws IOException {
+    return get(admin.getRegions(serverName));
+  }
+
+  @Override
+  public void flush(TableName tableName) throws IOException {
+    get(admin.flush(tableName));
+  }
+
+  @Override
+  public void flushRegion(byte[] regionName) throws IOException {
+    get(admin.flushRegion(regionName));
+  }
+
+  @Override
+  public void flushRegionServer(ServerName serverName) throws IOException {
+    get(admin.flushRegionServer(serverName));
+  }
+
+  @Override
+  public void compact(TableName tableName) throws IOException {
+    get(admin.compact(tableName));
+  }
+
+  @Override
+  public void compactRegion(byte[] regionName) throws IOException {
+    get(admin.compactRegion(regionName));
+  }
+
+  @Override
+  public void compact(TableName tableName, byte[] columnFamily) throws IOException {
+    get(admin.compact(tableName, columnFamily));
+  }
+
+  @Override
+  public void compactRegion(byte[] regionName, byte[] columnFamily) throws IOException {
+    get(admin.compactRegion(regionName, columnFamily));
+  }
+
+  @Override
+  public void compact(TableName tableName, CompactType compactType)
+      throws IOException, InterruptedException {
+    get(admin.compact(tableName, compactType));
+  }
+
+  @Override
+  public void compact(TableName tableName, byte[] columnFamily, CompactType compactType)
+      throws IOException, InterruptedException {
+    get(admin.compact(tableName, columnFamily, compactType));
+  }
+
+  @Override
+  public void majorCompact(TableName tableName) throws IOException {
+    get(admin.majorCompact(tableName));
+  }
+
+  @Override
+  public void majorCompactRegion(byte[] regionName) throws IOException {
+    get(admin.majorCompactRegion(regionName));
+  }
+
+  @Override
+  public void majorCompact(TableName tableName, byte[] columnFamily) throws IOException {
+    get(admin.majorCompact(tableName, columnFamily));
+  }
+
+  @Override
+  public void majorCompactRegion(byte[] regionName, byte[] columnFamily) throws IOException {
+    get(admin.majorCompactRegion(regionName, columnFamily));
+  }
+
+  @Override
+  public void majorCompact(TableName tableName, CompactType compactType)
+      throws IOException, InterruptedException {
+    get(admin.majorCompact(tableName, compactType));
+  }
+
+  @Override
+  public void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType)
+      throws IOException, InterruptedException {
+    get(admin.majorCompact(tableName, columnFamily, compactType));
+  }
+
+  @Override
+  public Map<ServerName, Boolean> compactionSwitch(boolean switchState,
+      List<String> serverNamesList) throws IOException {
+    return get(admin.compactionSwitch(switchState, serverNamesList));
+  }
+
+  @Override
+  public void compactRegionServer(ServerName serverName) throws IOException {
+    get(admin.compactRegionServer(serverName));
+  }
+
+  @Override
+  public void majorCompactRegionServer(ServerName serverName) throws IOException {
+    get(admin.majorCompactRegionServer(serverName));
+  }
+
+  @Override
+  public void move(byte[] encodedRegionName) throws IOException {
+    get(admin.move(encodedRegionName));
+  }
+
+  @Override
+  public void move(byte[] encodedRegionName, ServerName destServerName) throws IOException {
+    get(admin.move(encodedRegionName, destServerName));
+  }
+
+  @Override
+  public void assign(byte[] regionName) throws IOException {
+    get(admin.assign(regionName));
+  }
+
+  @Override
+  public void unassign(byte[] regionName, boolean force) throws IOException {
+    get(admin.unassign(regionName, force));
+  }
+
+  @Override
+  public void offline(byte[] regionName) throws IOException {
+    get(admin.offline(regionName));
+  }
+
+  @Override
+  public boolean balancerSwitch(boolean onOrOff, boolean synchronous) throws IOException {
+    return get(admin.balancerSwitch(onOrOff, synchronous));
+  }
+
+  @Override
+  public boolean balance() throws IOException {
+    return get(admin.balance());
+  }
+
+  @Override
+  public boolean balance(boolean force) throws IOException {
+    return get(admin.balance(force));
+  }
+
+  @Override
+  public boolean isBalancerEnabled() throws IOException {
+    return get(admin.isBalancerEnabled());
+  }
+
+  @Override
+  public CacheEvictionStats clearBlockCache(TableName tableName) throws IOException {
+    return get(admin.clearBlockCache(tableName));
+  }
+
+  @Override
+  public boolean normalize() throws IOException {
+    return get(admin.normalize());
+  }
+
+  @Override
+  public boolean isNormalizerEnabled() throws IOException {
+    return get(admin.isNormalizerEnabled());
+  }
+
+  @Override
+  public boolean normalizerSwitch(boolean on) throws IOException {
+    return get(admin.normalizerSwitch(on));
+  }
+
+  @Override
+  public boolean catalogJanitorSwitch(boolean onOrOff) throws IOException {
+    return get(admin.catalogJanitorSwitch(onOrOff));
+  }
+
+  @Override
+  public int runCatalogJanitor() throws IOException {
+    return get(admin.runCatalogJanitor());
+  }
+
+  @Override
+  public boolean isCatalogJanitorEnabled() throws IOException {
+    return get(admin.isCatalogJanitorEnabled());
+  }
+
+  @Override
+  public boolean cleanerChoreSwitch(boolean onOrOff) throws IOException {
+    return get(admin.cleanerChoreSwitch(onOrOff));
+  }
+
+  @Override
+  public boolean runCleanerChore() throws IOException {
+    return get(admin.runCleanerChore());
+  }
+
+  @Override
+  public boolean isCleanerChoreEnabled() throws IOException {
+    return get(admin.isCleanerChoreEnabled());
+  }
+
+  @Override
+  public Future<Void> mergeRegionsAsync(byte[][] nameOfRegionsToMerge, boolean forcible)
+      throws IOException {
+    return admin.mergeRegions(Arrays.asList(nameOfRegionsToMerge), forcible);
+  }
+
+  @Override
+  public void split(TableName tableName) throws IOException {
+    get(admin.split(tableName));
+  }
+
+  @Override
+  public void split(TableName tableName, byte[] splitPoint) throws IOException {
+    get(admin.split(tableName, splitPoint));
+  }
+
+  @Override
+  public Future<Void> splitRegionAsync(byte[] regionName) throws IOException {
+    return admin.splitRegion(regionName);
+  }
+
+  @Override
+  public Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException {
+    return admin.splitRegion(regionName, splitPoint);
+  }
+
+  @Override
+  public Future<Void> modifyTableAsync(TableDescriptor td) throws IOException {
+    return admin.modifyTable(td);
+  }
+
+  @Override
+  public void shutdown() throws IOException {
+    get(admin.shutdown());
+  }
+
+  @Override
+  public void stopMaster() throws IOException {
+    get(admin.stopMaster());
+  }
+
+  @Override
+  public boolean isMasterInMaintenanceMode() throws IOException {
+    return get(admin.isMasterInMaintenanceMode());
+  }
+
+  @Override
+  public void stopRegionServer(String hostnamePort) throws IOException {
+    get(admin.stopRegionServer(ServerName.valueOf(hostnamePort, 0)));
+  }
+
+  @Override
+  public ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException {
+    return get(admin.getClusterMetrics(options));
+  }
+
+  @Override
+  public List<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException {
+    return get(admin.getRegionMetrics(serverName));
+  }
+
+  @Override
+  public List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName)
+      throws IOException {
+    return get(admin.getRegionMetrics(serverName, tableName));
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return conn.getConfiguration();
+  }
+
+  @Override
+  public Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor) throws IOException {
+    return admin.createNamespace(descriptor);
+  }
+
+  @Override
+  public Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) throws IOException {
+    return admin.modifyNamespace(descriptor);
+  }
+
+  @Override
+  public Future<Void> deleteNamespaceAsync(String name) throws IOException {
+    return admin.deleteNamespace(name);
+  }
+
+  @Override
+  public NamespaceDescriptor getNamespaceDescriptor(String name)
+      throws NamespaceNotFoundException, IOException {
+    return get(admin.getNamespaceDescriptor(name));
+  }
+
+  @Override
+  public String[] listNamespaces() throws IOException {
+    return get(admin.listNamespaces()).toArray(new String[0]);
+  }
+
+  @Override
+  public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
+    return get(admin.listNamespaceDescriptors()).toArray(new NamespaceDescriptor[0]);
+  }
+
+  @Override
+  public List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException {
+    return get(admin.listTableDescriptorsByNamespace(Bytes.toString(name)));
+  }
+
+  @Override
+  public TableName[] listTableNamesByNamespace(String name) throws IOException {
+    return get(admin.listTableNamesByNamespace(name)).toArray(new TableName[0]);
+  }
+
+  @Override
+  public List<RegionInfo> getRegions(TableName tableName) throws IOException {
+    return get(admin.getRegions(tableName));
+  }
+
+  @Override
+  public void close() {
+    // do nothing, AsyncAdmin is not a Closeable.
+  }
+
+  @Override
+  public List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException {
+    return get(admin.listTableDescriptors(tableNames));
+  }
+
+  @Override
+  public Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning)
+      throws IOException {
+    return admin.abortProcedure(procId, mayInterruptIfRunning);
+  }
+
+  @Override
+  public String getProcedures() throws IOException {
+    return get(admin.getProcedures());
+  }
+
+  @Override
+  public String getLocks() throws IOException {
+    return get(admin.getLocks());
+  }
+
+  @Override
+  public void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException {
+    get(admin.rollWALWriter(serverName));
+  }
+
+  @Override
+  public CompactionState getCompactionState(TableName tableName) throws IOException {
+    return get(admin.getCompactionState(tableName));
+  }
+
+  @Override
+  public CompactionState getCompactionState(TableName tableName, CompactType compactType)
+      throws IOException {
+    return get(admin.getCompactionState(tableName, compactType));
+  }
+
+  @Override
+  public CompactionState getCompactionStateForRegion(byte[] regionName) throws IOException {
+    return get(admin.getCompactionStateForRegion(regionName));
+  }
+
+  @Override
+  public long getLastMajorCompactionTimestamp(TableName tableName) throws IOException {
+    return get(admin.getLastMajorCompactionTimestamp(tableName)).orElse(0L);
+  }
+
+  @Override
+  public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
+    return get(admin.getLastMajorCompactionTimestampForRegion(regionName)).orElse(0L);
+  }
+
+  @Override
+  public void snapshot(SnapshotDescription snapshot)
+      throws IOException, SnapshotCreationException, IllegalArgumentException {
+    get(admin.snapshot(snapshot));
+  }
+
+  @Override
+  public Future<Void> snapshotAsync(SnapshotDescription snapshot)
+      throws IOException, SnapshotCreationException {
+    return admin.snapshot(snapshot);
+  }
+
+  @Override
+  public boolean isSnapshotFinished(SnapshotDescription snapshot)
+      throws IOException, HBaseSnapshotException, UnknownSnapshotException {
+    return get(admin.isSnapshotFinished(snapshot));
+  }
+
+  @Override
+  public void restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException {
+    get(admin.restoreSnapshot(snapshotName));
+  }
+
+  @Override
+  public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl)
+      throws IOException, RestoreSnapshotException {
+    get(admin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl));
+  }
+
+  @Override
+  public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
+      boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException {
+    return admin.cloneSnapshot(snapshotName, tableName, restoreAcl);
+  }
+
+  @Override
+  public void execProcedure(String signature, String instance, Map<String, String> props)
+      throws IOException {
+    get(admin.execProcedure(signature, instance, props));
+  }
+
+  @Override
+  public byte[] execProcedureWithReturn(String signature, String instance,
+      Map<String, String> props) throws IOException {
+    return get(admin.execProcedureWithReturn(signature, instance, props));
+  }
+
+  @Override
+  public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
+      throws IOException {
+    return get(admin.isProcedureFinished(signature, instance, props));
+  }
+
+  @Override
+  public List<SnapshotDescription> listSnapshots() throws IOException {
+    return get(admin.listSnapshots());
+  }
+
+  @Override
+  public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
+    return get(admin.listSnapshots(pattern));
+  }
+
+  @Override
+  public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
+      Pattern snapshotNamePattern) throws IOException {
+    return get(admin.listTableSnapshots(tableNamePattern, snapshotNamePattern));
+  }
+
+  @Override
+  public void deleteSnapshot(String snapshotName) throws IOException {
+    get(admin.deleteSnapshot(snapshotName));
+  }
+
+  @Override
+  public void deleteSnapshots(Pattern pattern) throws IOException {
+    get(admin.deleteSnapshots(pattern));
+  }
+
+  @Override
+  public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
+      throws IOException {
+    get(admin.deleteTableSnapshots(tableNamePattern, snapshotNamePattern));
+  }
+
+  @Override
+  public void setQuota(QuotaSettings quota) throws IOException {
+    get(admin.setQuota(quota));
+  }
+
+  @Override
+  public List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException {
+    return get(admin.getQuota(filter));
+  }
+
+  @SuppressWarnings("deprecation")
+  private static final class SyncCoprocessorRpcChannelOverAsync implements CoprocessorRpcChannel {
+
+    private final RpcChannel delegate;
+
+    public SyncCoprocessorRpcChannelOverAsync(RpcChannel delegate) {
+      this.delegate = delegate;
+    }
+
+    @Override
+    public void callMethod(MethodDescriptor method, RpcController controller, Message request,
+        Message responsePrototype, RpcCallback<Message> done) {
+      ClientCoprocessorRpcController c = new ClientCoprocessorRpcController();
+      CoprocessorBlockingRpcCallback<Message> callback = new CoprocessorBlockingRpcCallback<>();
+      delegate.callMethod(method, c, request, responsePrototype, callback);
+      Message ret;
+      try {
+        ret = callback.get();
+      } catch (IOException e) {
+        setCoprocessorError(controller, e);
+        return;
+      }
+      if (c.failed()) {
+        setCoprocessorError(controller, c.getFailed());
+      }
+      done.run(ret);
+    }
+
+    @Override
+    public Message callBlockingMethod(MethodDescriptor method, RpcController controller,
+        Message request, Message responsePrototype) throws ServiceException {
+      ClientCoprocessorRpcController c = new ClientCoprocessorRpcController();
+      CoprocessorBlockingRpcCallback<Message> done = new CoprocessorBlockingRpcCallback<>();
+      callMethod(method, c, request, responsePrototype, done);
+      Message ret;
+      try {
+        ret = done.get();
+      } catch (IOException e) {
+        throw new ServiceException(e);
+      }
+      if (c.failed()) {
+        setCoprocessorError(controller, c.getFailed());
+        throw new ServiceException(c.getFailed());
+      }
+      return ret;
+    }
+  }
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public CoprocessorRpcChannel coprocessorService() {
+    return new SyncCoprocessorRpcChannelOverAsync(
+      new MasterCoprocessorRpcChannelImpl(admin.<Message> newMasterCaller()));
+  }
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public CoprocessorRpcChannel coprocessorService(ServerName serverName) {
+    return new SyncCoprocessorRpcChannelOverAsync(new RegionServerCoprocessorRpcChannelImpl(
+      admin.<Message> newServerCaller().serverName(serverName)));
+  }
+
+  @Override
+  public void updateConfiguration(ServerName server) throws IOException {
+    get(admin.updateConfiguration(server));
+  }
+
+  @Override
+  public void updateConfiguration() throws IOException {
+    get(admin.updateConfiguration());
+  }
+
+  @Override
+  public List<SecurityCapability> getSecurityCapabilities() throws IOException {
+    return get(admin.getSecurityCapabilities());
+  }
+
+  @Override
+  public boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException {
+    return get(admin.splitSwitch(enabled, synchronous));
+  }
+
+  @Override
+  public boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException {
+    return get(admin.mergeSwitch(enabled, synchronous));
+  }
+
+  @Override
+  public boolean isSplitEnabled() throws IOException {
+    return get(admin.isSplitEnabled());
+  }
+
+  @Override
+  public boolean isMergeEnabled() throws IOException {
+    return get(admin.isMergeEnabled());
+  }
+
+  @Override
+  public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
+      boolean enabled) throws IOException {
+    return admin.addReplicationPeer(peerId, peerConfig, enabled);
+  }
+
+  @Override
+  public Future<Void> removeReplicationPeerAsync(String peerId) throws IOException {
+    return admin.removeReplicationPeer(peerId);
+  }
+
+  @Override
+  public Future<Void> enableReplicationPeerAsync(String peerId) throws IOException {
+    return admin.enableReplicationPeer(peerId);
+  }
+
+  @Override
+  public Future<Void> disableReplicationPeerAsync(String peerId) throws IOException {
+    return admin.disableReplicationPeer(peerId);
+  }
+
+  @Override
+  public ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws IOException {
+    return get(admin.getReplicationPeerConfig(peerId));
+  }
+
+  @Override
+  public Future<Void> updateReplicationPeerConfigAsync(String peerId,
+      ReplicationPeerConfig peerConfig) throws IOException {
+    return admin.updateReplicationPeerConfig(peerId, peerConfig);
+  }
+
+  @Override
+  public List<ReplicationPeerDescription> listReplicationPeers() throws IOException {
+    return get(admin.listReplicationPeers());
+  }
+
+  @Override
+  public List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException {
+    return get(admin.listReplicationPeers(pattern));
+  }
+
+  @Override
+  public Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
+      SyncReplicationState state) throws IOException {
+    return admin.transitReplicationPeerSyncReplicationState(peerId, state);
+  }
+
+  @Override
+  public void decommissionRegionServers(List<ServerName> servers, boolean offload)
+      throws IOException {
+    get(admin.decommissionRegionServers(servers, offload));
+  }
+
+  @Override
+  public List<ServerName> listDecommissionedRegionServers() throws IOException {
+    return get(admin.listDecommissionedRegionServers());
+  }
+
+  @Override
+  public void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
+      throws IOException {
+    get(admin.recommissionRegionServer(server, encodedRegionNames));
+  }
+
+  @Override
+  public List<TableCFs> listReplicatedTableCFs() throws IOException {
+    return get(admin.listReplicatedTableCFs());
+  }
+
+  @Override
+  public void enableTableReplication(TableName tableName) throws IOException {
+    get(admin.enableTableReplication(tableName));
+  }
+
+  @Override
+  public void disableTableReplication(TableName tableName) throws IOException {
+    get(admin.disableTableReplication(tableName));
+  }
+
+  @Override
+  public void clearCompactionQueues(ServerName serverName, Set<String> queues)
+      throws IOException, InterruptedException {
+    get(admin.clearCompactionQueues(serverName, queues));
+  }
+
+  @Override
+  public List<ServerName> clearDeadServers(List<ServerName> servers) throws IOException {
+    return get(admin.clearDeadServers(servers));
+  }
+
+  @Override
+  public void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits)
+      throws IOException {
+    get(admin.cloneTableSchema(tableName, newTableName, preserveSplits));
+  }
+
+  @Override
+  public boolean switchRpcThrottle(boolean enable) throws IOException {
+    return get(admin.switchRpcThrottle(enable));
+  }
+
+  @Override
+  public boolean isRpcThrottleEnabled() throws IOException {
+    return get(admin.isRpcThrottleEnabled());
+  }
+
+  @Override
+  public boolean exceedThrottleQuotaSwitch(boolean enable) throws IOException {
+    return get(admin.exceedThrottleQuotaSwitch(enable));
+  }
+
+  @Override
+  public Map<TableName, Long> getSpaceQuotaTableSizes() throws IOException {
+    return get(admin.getSpaceQuotaTableSizes());
+  }
+
+  @Override
+  public Map<TableName, ? extends SpaceQuotaSnapshotView> getRegionServerSpaceQuotaSnapshots(
+      ServerName serverName) throws IOException {
+    return get(admin.getRegionServerSpaceQuotaSnapshots(serverName));
+  }
+
+  @Override
+  public SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(String namespace) throws IOException {
+    return get(admin.getCurrentSpaceQuotaSnapshot(namespace));
+  }
+
+  @Override
+  public SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(TableName tableName)
+      throws IOException {
+    return get(admin.getCurrentSpaceQuotaSnapshot(tableName));
+  }
+
+  @Override
+  public void grant(UserPermission userPermission, boolean mergeExistingPermissions)
+      throws IOException {
+    get(admin.grant(userPermission, mergeExistingPermissions));
+  }
+
+  @Override
+  public void revoke(UserPermission userPermission) throws IOException {
+    get(admin.revoke(userPermission));
+  }
+
+  @Override
+  public List<UserPermission> getUserPermissions(
+      GetUserPermissionsRequest getUserPermissionsRequest) throws IOException {
+    return get(admin.getUserPermissions(getUserPermissionsRequest));
+  }
+
+  @Override
+  public List<Boolean> hasUserPermissions(String userName, List<Permission> permissions)
+      throws IOException {
+    return get(admin.hasUserPermissions(userName, permissions));
+  }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java
index 61cc708..dfe7d8f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java
@@ -102,7 +102,7 @@ class ConnectionOverAsyncConnection implements Connection {
 
   @Override
   public Admin getAdmin() throws IOException {
-    return oldConn.getAdmin();
+    return new AdminOverAsyncAdmin(this, (RawAsyncHBaseAdmin) conn.getAdmin());
   }
 
   @Override
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 2fa30b5..a6c47b5 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
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.ServerName;
 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;
@@ -715,4 +716,21 @@ public final class ConnectionUtils {
       pool.shutdownNow();
     }
   }
+
+  static void setCoprocessorError(com.google.protobuf.RpcController controller, Throwable error) {
+    if (controller == null) {
+      return;
+    }
+    if (controller instanceof ServerRpcController) {
+      if (error instanceof IOException) {
+        ((ServerRpcController) controller).setFailedOn((IOException) error);
+      } else {
+        ((ServerRpcController) controller).setFailedOn(new IOException(error));
+      }
+    } else if (controller instanceof ClientCoprocessorRpcController) {
+      ((ClientCoprocessorRpcController) controller).setFailed(error);
+    } else {
+      controller.setFailed(error.toString());
+    }
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CoprocessorBlockingRpcCallback.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CoprocessorBlockingRpcCallback.java
new file mode 100644
index 0000000..30f6e7e
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CoprocessorBlockingRpcCallback.java
@@ -0,0 +1,68 @@
+/**
+ * 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.RpcCallback;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * For implementation coprocessor related methods in {@link Table} and {@link Admin} interface.
+ * @deprecated since 3.0.0, will be removed in 4.0.0 along with the coprocessor related methods in
+ *             {@link Table} and {@link Admin} interface.
+ */
+@Deprecated
+@InterfaceAudience.Private
+class CoprocessorBlockingRpcCallback<R> implements RpcCallback<R> {
+  private R result;
+  private boolean resultSet = false;
+
+  /**
+   * Called on completion of the RPC call with the response object, or {@code null} in the case of
+   * an error.
+   * @param parameter the response object or {@code null} if an error occurred
+   */
+  @Override
+  public void run(R parameter) {
+    synchronized (this) {
+      result = parameter;
+      resultSet = true;
+      this.notifyAll();
+    }
+  }
+
+  /**
+   * Returns the parameter passed to {@link #run(Object)} or {@code null} if a null value was
+   * passed. When used asynchronously, this method will block until the {@link #run(Object)} method
+   * has been called.
+   * @return the response object or {@code null} if no response was passed
+   */
+  public synchronized R get() throws IOException {
+    while (!resultSet) {
+      try {
+        this.wait();
+      } catch (InterruptedException ie) {
+        InterruptedIOException exception = new InterruptedIOException(ie.getMessage());
+        exception.initCause(ie);
+        throw exception;
+      }
+    }
+    return result;
+  }
+}
\ No newline at end of file
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index c466e61..9c62678 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -1156,10 +1156,10 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public void move(byte[] encodedRegionName) throws IOException {
-    move(encodedRegionName, (ServerName) null);
+    move(encodedRegionName, null);
   }
 
-  public void move(final byte[] encodedRegionName, ServerName destServerName) throws IOException {
+  public void move(byte[] encodedRegionName, ServerName destServerName) throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
       protected Void rpcCall() throws Exception {
@@ -3910,6 +3910,11 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
+  public List<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException {
+    return getRegionMetrics(serverName, null);
+  }
+
+  @Override
   public Future<Void> createTableAsync(TableDescriptor desc) throws IOException {
     return createTableAsync(desc, null);
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index b3d3468..47a7902 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -360,7 +360,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     this.ng = connection.getNonceGenerator();
   }
 
-  private <T> MasterRequestCallerBuilder<T> newMasterCaller() {
+  <T> MasterRequestCallerBuilder<T> newMasterCaller() {
     return this.connection.callerFactory.<T> masterRequest()
       .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
       .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
@@ -702,11 +702,6 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   @Override
   public CompletableFuture<Boolean> isTableAvailable(TableName tableName) {
-    return isTableAvailable(tableName, Optional.empty());
-  }
-
-  private CompletableFuture<Boolean> isTableAvailable(TableName tableName,
-      Optional<byte[][]> splitKeys) {
     if (TableName.isMetaTableName(tableName)) {
       return connection.registry.getMetaRegionLocation().thenApply(locs -> Stream
         .of(locs.getRegionLocations()).allMatch(loc -> loc != null && loc.getServerName() != null));
@@ -740,35 +735,13 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
               future.complete(false);
               return;
             }
-
-            Optional<Boolean> available =
-              splitKeys.map(keys -> compareRegionsWithSplitKeys(locations, keys));
-            future.complete(available.orElse(true));
+            future.complete(true);
           });
       }
     });
     return future;
   }
 
-  private boolean compareRegionsWithSplitKeys(List<HRegionLocation> locations, byte[][] splitKeys) {
-    int regionCount = 0;
-    for (HRegionLocation location : locations) {
-      RegionInfo info = location.getRegion();
-      if (Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
-        regionCount++;
-        continue;
-      }
-      for (byte[] splitKey : splitKeys) {
-        // Just check if the splitkey is available
-        if (Bytes.equals(info.getStartKey(), splitKey)) {
-          regionCount++;
-          break;
-        }
-      }
-    }
-    return regionCount == splitKeys.length + 1;
-  }
-
   @Override
   public CompletableFuture<Void> addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) {
     return this.<AddColumnRequest, AddColumnResponse> procedureCall(tableName,
@@ -2004,10 +1977,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
                     LOG.error(
                       "Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName,
                       err3);
-                    future.completeExceptionally(err3);
-                  } else {
-                    future.complete(ret3);
                   }
+                  future.complete(ret3);
                 });
               }
             });
@@ -3393,7 +3364,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       .call();
   }
 
-  private <T> ServerRequestCallerBuilder<T> newServerCaller() {
+  <T> ServerRequestCallerBuilder<T> newServerCaller() {
     return this.connection.callerFactory.<T> serverRequest()
       .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
       .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java
index 3c25c57..b41727f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.client.ConnectionUtils.setCoprocessorError;
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import com.google.protobuf.Descriptors.MethodDescriptor;
@@ -32,7 +33,6 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -101,23 +101,6 @@ class RegionCoprocessorRpcChannelImpl implements RpcChannel {
     return future;
   }
 
-  protected final void setError(RpcController controller, Throwable error) {
-    if (controller == null) {
-      return;
-    }
-    if (controller instanceof ServerRpcController) {
-      if (error instanceof IOException) {
-        ((ServerRpcController) controller).setFailedOn((IOException) error);
-      } else {
-        ((ServerRpcController) controller).setFailedOn(new IOException(error));
-      }
-    } else if (controller instanceof ClientCoprocessorRpcController) {
-      ((ClientCoprocessorRpcController) controller).setFailed(error);
-    } else {
-      controller.setFailed(error.toString());
-    }
-  }
-
   @Override
   public void callMethod(MethodDescriptor method, RpcController controller, Message request,
       Message responsePrototype, RpcCallback<Message> done) {
@@ -128,7 +111,7 @@ class RegionCoprocessorRpcChannelImpl implements RpcChannel {
         .action((c, l, s) -> rpcCall(method, request, responsePrototype, c, l, s)).call(),
       (r, e) -> {
         if (e != null) {
-          setError(controller, e);
+          setCoprocessorError(controller, e);
         }
         done.run(r);
       });
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SyncCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SyncCoprocessorRpcChannel.java
index 6b4419d..2811219 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SyncCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SyncCoprocessorRpcChannel.java
@@ -36,7 +36,10 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
  * call coprocessor endpoint {@link com.google.protobuf.Service}s.
  * Note that clients should not use this class directly, except through
  * {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])}.
+ * @deprecated Please stop using this class again, as it is too low level, which is part of the rpc
+ *             framework for HBase. Will be deleted in 4.0.0.
  */
+@Deprecated
 @InterfaceAudience.Public
 abstract class SyncCoprocessorRpcChannel implements CoprocessorRpcChannel {
   private static final Logger LOG = LoggerFactory.getLogger(SyncCoprocessorRpcChannel.class);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java
index d581611..30e3062 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.client.ConnectionUtils.setCoprocessorError;
+
 import com.google.protobuf.Descriptors.MethodDescriptor;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcCallback;
@@ -298,44 +300,7 @@ class TableOverAsyncTable implements Table {
   public void close() {
   }
 
-  private static final class BlockingRpcCallback<R> implements RpcCallback<R> {
-    private R result;
-    private boolean resultSet = false;
-
-    /**
-     * Called on completion of the RPC call with the response object, or {@code null} in the case of
-     * an error.
-     * @param parameter the response object or {@code null} if an error occurred
-     */
-    @Override
-    public void run(R parameter) {
-      synchronized (this) {
-        result = parameter;
-        resultSet = true;
-        this.notifyAll();
-      }
-    }
-
-    /**
-     * Returns the parameter passed to {@link #run(Object)} or {@code null} if a null value was
-     * passed. When used asynchronously, this method will block until the {@link #run(Object)}
-     * method has been called.
-     * @return the response object or {@code null} if no response was passed
-     */
-    public synchronized R get() throws IOException {
-      while (!resultSet) {
-        try {
-          this.wait();
-        } catch (InterruptedException ie) {
-          InterruptedIOException exception = new InterruptedIOException(ie.getMessage());
-          exception.initCause(ie);
-          throw exception;
-        }
-      }
-      return result;
-    }
-  }
-
+  @SuppressWarnings("deprecation")
   private static final class RegionCoprocessorRpcChannel extends RegionCoprocessorRpcChannelImpl
       implements CoprocessorRpcChannel {
 
@@ -348,17 +313,17 @@ class TableOverAsyncTable implements Table {
     public void callMethod(MethodDescriptor method, RpcController controller, Message request,
         Message responsePrototype, RpcCallback<Message> done) {
       ClientCoprocessorRpcController c = new ClientCoprocessorRpcController();
-      BlockingRpcCallback<Message> callback = new BlockingRpcCallback<>();
+      CoprocessorBlockingRpcCallback<Message> callback = new CoprocessorBlockingRpcCallback<>();
       super.callMethod(method, c, request, responsePrototype, callback);
       Message ret;
       try {
         ret = callback.get();
       } catch (IOException e) {
-        setError(controller, e);
+        setCoprocessorError(controller, e);
         return;
       }
       if (c.failed()) {
-        setError(controller, c.getFailed());
+        setCoprocessorError(controller, c.getFailed());
       }
       done.run(ret);
     }
@@ -367,7 +332,7 @@ class TableOverAsyncTable implements Table {
     public Message callBlockingMethod(MethodDescriptor method, RpcController controller,
         Message request, Message responsePrototype) throws ServiceException {
       ClientCoprocessorRpcController c = new ClientCoprocessorRpcController();
-      BlockingRpcCallback<Message> done = new BlockingRpcCallback<>();
+      CoprocessorBlockingRpcCallback<Message> done = new CoprocessorBlockingRpcCallback<>();
       callMethod(method, c, request, responsePrototype, done);
       Message ret;
       try {
@@ -376,7 +341,7 @@ class TableOverAsyncTable implements Table {
         throw new ServiceException(e);
       }
       if (c.failed()) {
-        setError(controller, c.getFailed());
+        setCoprocessorError(controller, c.getFailed());
         throw new ServiceException(c.getFailed());
       }
       return ret;
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
index 953fba7..3c8b04d 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
@@ -57,6 +57,8 @@ public class TestInterfaceAlign {
     adminMethodNames.removeAll(getMethodNames(Abortable.class));
     adminMethodNames.removeAll(getMethodNames(Closeable.class));
 
+    asyncAdminMethodNames.remove("coprocessorService");
+
     adminMethodNames.forEach(method -> {
       boolean contains = asyncAdminMethodNames.contains(method);
       if (method.endsWith("Async")) {
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 2dae0e8..7972da0 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -383,7 +383,11 @@ public class PerformanceEvaluation extends Configured implements Tool {
           }
         }
       }
-      admin.createTable(desc, splits);
+      if (splits != null) {
+        admin.createTable(desc, splits);
+      } else {
+        admin.createTable(desc);
+      }
       LOG.info("Table " + desc + " created");
     }
     return admin.tableExists(tableName);
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
index 1d7a37c..d5247fb 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
@@ -600,8 +600,8 @@ public class TestRemoteTable {
     REST_TEST_UTIL.startServletContainer(TEST_UTIL.getConfiguration());
 
     // Truncate the test table for inserting test scenarios rows keys
-    TEST_UTIL.getHBaseAdmin().disableTable(TABLE);
-    TEST_UTIL.getHBaseAdmin().truncateTable(TABLE, false);
+    TEST_UTIL.getAdmin().disableTable(TABLE);
+    TEST_UTIL.getAdmin().truncateTable(TABLE, false);
 
     remoteTable = new RemoteHTable(
         new Client(new Cluster().add("localhost", REST_TEST_UTIL.getServletPort())),
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 58a3f10..afca997 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -74,7 +74,6 @@ import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Hbck;
 import org.apache.hadoop.hbase.client.ImmutableHRegionInfo;
 import org.apache.hadoop.hbase.client.ImmutableHTableDescriptor;
@@ -1590,7 +1589,11 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       builder.setColumnFamily(cfdb.build());
     }
     TableDescriptor td = builder.build();
-    getAdmin().createTable(td, splitKeys);
+    if (splitKeys != null) {
+      getAdmin().createTable(td, splitKeys);
+    } else {
+      getAdmin().createTable(td);
+    }
     // HBaseAdmin only waits for regions to appear in hbase:meta
     // we should wait until they are assigned
     waitUntilAllRegionsAssigned(td.getTableName());
@@ -1613,7 +1616,11 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
            .setNewVersionBehavior(true).build());
       }
     }
-    getAdmin().createTable(builder.build(), splitRows);
+    if (splitRows != null) {
+      getAdmin().createTable(builder.build(), splitRows);
+    } else {
+      getAdmin().createTable(builder.build());
+    }
     // HBaseAdmin only waits for regions to appear in hbase:meta
     // we should wait until they are assigned
     waitUntilAllRegionsAssigned(htd.getTableName());
@@ -1682,7 +1689,11 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       }
       desc.addFamily(hcd);
     }
-    getAdmin().createTable(desc, splitKeys);
+    if (splitKeys != null) {
+      getAdmin().createTable(desc, splitKeys);
+    } else {
+      getAdmin().createTable(desc);
+    }
     // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
     // assigned
     waitUntilAllRegionsAssigned(tableName);
@@ -3031,36 +3042,17 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   }
 
   /**
-   * Returns a Admin instance.
-   * This instance is shared between HBaseTestingUtility instance users. Closing it has no effect,
-   * it will be closed automatically when the cluster shutdowns
-   *
-   * @return HBaseAdmin instance which is guaranteed to support only {@link Admin} interface.
-   *   Functions in HBaseAdmin not provided by {@link Admin} interface can be changed/deleted
-   *   anytime.
-   * @deprecated Since 2.0. Will be removed in 3.0. Use {@link #getAdmin()} instead.
-   */
-  @Deprecated
-  public synchronized HBaseAdmin getHBaseAdmin()
-  throws IOException {
-    if (hbaseAdmin == null){
-      this.hbaseAdmin = (HBaseAdmin) getConnection().getAdmin();
-    }
-    return hbaseAdmin;
-  }
-
-  /**
    * Returns an Admin instance which is shared between HBaseTestingUtility instance users.
    * Closing it has no effect, it will be closed automatically when the cluster shutdowns
    */
   public synchronized Admin getAdmin() throws IOException {
     if (hbaseAdmin == null){
-      this.hbaseAdmin = (HBaseAdmin) getConnection().getAdmin();
+      this.hbaseAdmin = getConnection().getAdmin();
     }
     return hbaseAdmin;
   }
 
-  private HBaseAdmin hbaseAdmin = null;
+  private Admin hbaseAdmin = null;
 
   /**
    * Returns an {@link Hbck} instance. Needs be closed when done.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index 8e9afed..538917d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
@@ -1080,7 +1081,11 @@ public class TestAdmin1 {
 
       // Split the table
       if (async) {
-        ADMIN.split(tableName, splitPoint);
+        if (splitPoint != null) {
+          ADMIN.split(tableName, splitPoint);
+        } else {
+          ADMIN.split(tableName);
+        }
         final AtomicInteger count = new AtomicInteger(0);
         Thread t = new Thread("CheckForSplit") {
           @Override public void run() {
@@ -1205,7 +1210,8 @@ public class TestAdmin1 {
     // the element at index 1 would be a replica (since the metareader gives us ordered
     // regions). Try splitting that region via the split API . Should fail
     try {
-      TEST_UTIL.getAdmin().splitRegionAsync(regions.get(1).getFirst().getRegionName()).get();
+      FutureUtils.get(
+        TEST_UTIL.getAdmin().splitRegionAsync(regions.get(1).getFirst().getRegionName()));
     } catch (IllegalArgumentException ex) {
       gotException = true;
     }
@@ -1215,9 +1221,9 @@ public class TestAdmin1 {
     // regions). Try splitting that region via a different split API (the difference is
     // this API goes direct to the regionserver skipping any checks in the admin). Should fail
     try {
-      TEST_UTIL.getHBaseAdmin().splitRegionAsync(regions.get(1).getFirst(),
-          new byte[]{(byte)'1'});
-    } catch (IOException ex) {
+      FutureUtils.get(TEST_UTIL.getAdmin().splitRegionAsync(
+        regions.get(1).getFirst().getEncodedNameAsBytes(), new byte[] { (byte) '1' }));
+    } catch (IllegalArgumentException ex) {
       gotException = true;
     }
     assertTrue(gotException);
@@ -1225,8 +1231,8 @@ public class TestAdmin1 {
     gotException = false;
     //testing Sync split operation
     try {
-      TEST_UTIL.getHBaseAdmin().splitRegionSync(regions.get(1).getFirst().getRegionName(),
-          new byte[]{(byte)'1'});
+      FutureUtils.get(TEST_UTIL.getAdmin()
+        .splitRegionAsync(regions.get(1).getFirst().getRegionName(), new byte[] { (byte) '1' }));
     } catch (IllegalArgumentException ex) {
       gotException = true;
     }
@@ -1235,10 +1241,10 @@ public class TestAdmin1 {
     gotException = false;
     // Try merging a replica with another. Should fail.
     try {
-      TEST_UTIL.getHBaseAdmin().mergeRegionsSync(
+      FutureUtils.get(TEST_UTIL.getAdmin().mergeRegionsAsync(
         regions.get(1).getFirst().getEncodedNameAsBytes(),
         regions.get(2).getFirst().getEncodedNameAsBytes(),
-        true);
+        true));
     } catch (IllegalArgumentException m) {
       gotException = true;
     }
@@ -1246,12 +1252,12 @@ public class TestAdmin1 {
     // Try going to the master directly (that will skip the check in admin)
     try {
       byte[][] nameofRegionsToMerge = new byte[2][];
-      nameofRegionsToMerge[0] =  regions.get(1).getFirst().getEncodedNameAsBytes();
+      nameofRegionsToMerge[0] = regions.get(1).getFirst().getEncodedNameAsBytes();
       nameofRegionsToMerge[1] = regions.get(2).getFirst().getEncodedNameAsBytes();
       MergeTableRegionsRequest request = RequestConverter.buildMergeTableRegionsRequest(
         nameofRegionsToMerge, true, HConstants.NO_NONCE, HConstants.NO_NONCE);
-      ((ConnectionImplementation) TEST_UTIL.getAdmin().getConnection()).getMaster()
-        .mergeTableRegions(null, request);
+      TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterRpcServices().mergeTableRegions(null,
+        request);
     } catch (org.apache.hbase.thirdparty.com.google.protobuf.ServiceException m) {
       Throwable t = m.getCause();
       do {
@@ -1439,24 +1445,24 @@ public class TestAdmin1 {
       List<RegionInfo> tableRegions = ADMIN.getRegions(tableName);
       // 0
       try {
-        ADMIN.mergeRegionsAsync(new byte[0][0], false).get();
+        FutureUtils.get(ADMIN.mergeRegionsAsync(new byte[0][0], false));
         fail();
       } catch (IllegalArgumentException e) {
         // expected
       }
       // 1
       try {
-        ADMIN.mergeRegionsAsync(new byte[][] { tableRegions.get(0).getEncodedNameAsBytes() }, false)
-          .get();
+        FutureUtils.get(ADMIN
+          .mergeRegionsAsync(new byte[][] { tableRegions.get(0).getEncodedNameAsBytes() }, false));
         fail();
       } catch (IllegalArgumentException e) {
         // expected
       }
       // 3
       try {
-        ADMIN.mergeRegionsAsync(
+        FutureUtils.get(ADMIN.mergeRegionsAsync(
           tableRegions.stream().map(RegionInfo::getEncodedNameAsBytes).toArray(byte[][]::new),
-          false).get();
+          false));
         fail();
       } catch (DoNotRetryIOException e) {
         // expected
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index 58a8bc5..2644061 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
@@ -59,7 +58,6 @@ import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -211,7 +209,7 @@ public class TestAdmin2 {
       // Use 80 bit numbers to make sure we aren't limited
       byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 };
       byte [] endKey =   { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
-      Admin hbaseadmin = TEST_UTIL.getHBaseAdmin();
+      Admin hbaseadmin = TEST_UTIL.getAdmin();
       HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
       htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
       hbaseadmin.createTable(htd, startKey, endKey, expectedRegions);
@@ -391,14 +389,14 @@ public class TestAdmin2 {
       isInList);
   }
 
-  private HBaseAdmin createTable(TableName tableName) throws IOException {
-    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+  private Admin createTable(TableName tableName) throws IOException {
+    Admin admin = TEST_UTIL.getAdmin();
 
     HTableDescriptor htd = new HTableDescriptor(tableName);
     HColumnDescriptor hcd = new HColumnDescriptor("value");
 
     htd.addFamily(hcd);
-    admin.createTable(htd, null);
+    admin.createTable(htd);
     return admin;
   }
 
@@ -411,7 +409,7 @@ public class TestAdmin2 {
     HColumnDescriptor hcd = new HColumnDescriptor("value");
     htd.addFamily(hcd);
 
-    ADMIN.createTable(htd, null);
+    ADMIN.createTable(htd);
   }
 
   /**
@@ -588,7 +586,7 @@ public class TestAdmin2 {
         new HTableDescriptor(TableName.valueOf(Bytes.toBytes(name.getMethodName())));
     HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("cf1"));
     htd.addFamily(hcd);
-    TEST_UTIL.getHBaseAdmin().createTable(htd);
+    TEST_UTIL.getAdmin().createTable(htd);
   }
 
   @Test
@@ -607,27 +605,6 @@ public class TestAdmin2 {
   }
 
   @Test
-  public void testGetRegion() throws Exception {
-    // We use actual HBaseAdmin instance instead of going via Admin interface in
-    // here because makes use of an internal HBA method (TODO: Fix.).
-    HBaseAdmin rawAdmin = TEST_UTIL.getHBaseAdmin();
-
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    LOG.info("Started " + tableName);
-    Table t = TEST_UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY);
-
-    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
-      HRegionLocation regionLocation = locator.getRegionLocation(Bytes.toBytes("mmm"));
-      RegionInfo region = regionLocation.getRegionInfo();
-      byte[] regionName = region.getRegionName();
-      Pair<RegionInfo, ServerName> pair = rawAdmin.getRegion(regionName);
-      assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
-      pair = rawAdmin.getRegion(region.getEncodedNameAsBytes());
-      assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
-    }
-  }
-
-  @Test
   public void testBalancer() throws Exception {
     boolean initialState = ADMIN.isBalancerEnabled();
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 562ca37..680c90a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -4249,8 +4249,7 @@ public class TestFromClientSide {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
     try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
-      try (Table t = conn.getTable(tableName);
-           Admin admin = conn.getAdmin()) {
+      try (Table t = conn.getTable(tableName); Admin admin = conn.getAdmin()) {
         assertTrue(admin.tableExists(tableName));
         assertTrue(t.get(new Get(ROW)).isEmpty());
       }
@@ -4269,8 +4268,8 @@ public class TestFromClientSide {
       boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
       try (Admin admin = conn.getAdmin()) {
         assertTrue(admin.tableExists(tableName));
-        assertTrue(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
-                .getLiveServerMetrics().size() == SLAVES + (tablesOnMaster ? 1 : 0));
+        assertTrue(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics()
+          .size() == SLAVES + (tablesOnMaster ? 1 : 0));
       }
     }
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index 83becbc..3b3f636 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
@@ -272,13 +272,13 @@ public class TestFromClientSide3 {
     TEST_UTIL.getConfiguration().setInt("hbase.hstore.compaction.min", 3);
 
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10)) {
+    try (Table table = TEST_UTIL.createTable(tableName, FAMILY, 10)) {
       TEST_UTIL.waitTableAvailable(tableName, WAITTABLE_MILLIS);
       Admin admin = TEST_UTIL.getAdmin();
 
       // Create 3 store files.
       byte[] row = Bytes.toBytes(random.nextInt());
-      performMultiplePutAndFlush(admin, hTable, row, FAMILY, 3, 100);
+      performMultiplePutAndFlush(admin, table, row, FAMILY, 3, 100);
 
       try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
         // Verify we have multiple store files.
@@ -304,13 +304,13 @@ public class TestFromClientSide3 {
 
         // change the compaction.min config option for this table to 5
         LOG.info("hbase.hstore.compaction.min should now be 5");
-        HTableDescriptor htd = new HTableDescriptor(hTable.getDescriptor());
+        HTableDescriptor htd = new HTableDescriptor(table.getDescriptor());
         htd.setValue("hbase.hstore.compaction.min", String.valueOf(5));
         admin.modifyTable(htd);
         LOG.info("alter status finished");
 
         // Create 3 more store files.
-        performMultiplePutAndFlush(admin, hTable, row, FAMILY, 3, 10);
+        performMultiplePutAndFlush(admin, table, row, FAMILY, 3, 10);
 
         // Issue a compaction request
         admin.compact(tableName);
@@ -357,7 +357,7 @@ public class TestFromClientSide3 {
         htd.modifyFamily(hcd);
         admin.modifyTable(htd);
         LOG.info("alter status finished");
-        assertNull(hTable.getDescriptor().getColumnFamily(FAMILY)
+        assertNull(table.getDescriptor().getColumnFamily(FAMILY)
           .getValue(Bytes.toBytes("hbase.hstore.compaction.min")));
       }
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotDFSTemporaryDirectory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotDFSTemporaryDirectory.java
index b4cef33..7501867 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotDFSTemporaryDirectory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotDFSTemporaryDirectory.java
@@ -50,10 +50,11 @@ public class TestSnapshotDFSTemporaryDirectory
    *
    * @throws Exception on failure
    */
-  @BeforeClass public static void setupCluster() throws Exception {
+  @BeforeClass
+  public static void setupCluster() throws Exception {
     setupConf(UTIL.getConfiguration());
     UTIL.startMiniCluster(NUM_RS);
-    admin = UTIL.getHBaseAdmin();
+    admin = UTIL.getAdmin();
   }
 
   private static void setupConf(Configuration conf) throws IOException {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
index a8561d0..a945612 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
@@ -100,10 +100,11 @@ public class TestSnapshotTemporaryDirectory {
    *
    * @throws Exception on failure
    */
-  @BeforeClass public static void setupCluster() throws Exception {
+  @BeforeClass
+  public static void setupCluster() throws Exception {
     setupConf(UTIL.getConfiguration());
     UTIL.startMiniCluster(NUM_RS);
-    admin = UTIL.getHBaseAdmin();
+    admin = UTIL.getAdmin();
   }
 
   private static void setupConf(Configuration conf) {
@@ -136,7 +137,7 @@ public class TestSnapshotTemporaryDirectory {
 
   @After public void tearDown() throws Exception {
     UTIL.deleteTable(TABLE_NAME);
-    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getAdmin());
     SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
   }
 
@@ -282,8 +283,9 @@ public class TestSnapshotTemporaryDirectory {
    *
    * @throws Exception if snapshot does not complete successfully
    */
-  @Test(timeout = 300000) public void testOfflineTableSnapshot() throws Exception {
-    Admin admin = UTIL.getHBaseAdmin();
+  @Test(timeout = 300000)
+  public void testOfflineTableSnapshot() throws Exception {
+    Admin admin = UTIL.getAdmin();
     // make sure we don't fail on listing snapshots
     SnapshotTestingUtils.assertNoSnapshots(admin);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
index 3e40b6f..9a55838 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
@@ -55,17 +55,11 @@ public class TestSplitOrMergeStatus {
   @Rule
   public TestName name = new TestName();
 
-  /**
-   * @throws java.lang.Exception
-   */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.startMiniCluster(2);
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
@@ -84,7 +78,12 @@ public class TestSplitOrMergeStatus {
     initSwitchStatus(admin);
     boolean result = admin.splitSwitch(false, false);
     assertTrue(result);
-    admin.split(t.getName());
+    try {
+      admin.split(t.getName());
+      fail();
+    } catch (IOException e) {
+      // expected
+    }
     int count = admin.getRegions(tableName).size();
     assertTrue(originalCount == count);
     result = admin.splitSwitch(true, false);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 1bc3996..76618a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -1342,7 +1342,7 @@ public class TestMasterObserver {
       List<HRegionLocation> regions = regionLocator.getAllRegionLocations();
 
       admin.mergeRegionsAsync(regions.get(0).getRegionInfo().getEncodedNameAsBytes(),
-        regions.get(1).getRegionInfo().getEncodedNameAsBytes(), true);
+        regions.get(1).getRegionInfo().getEncodedNameAsBytes(), true).get();
       assertTrue("Coprocessor should have been called on region merge",
         cp.wasMergeRegionsCalled());
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
index 7c396c7..fa23d38 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
@@ -190,7 +190,7 @@ public class TestMaster {
     HColumnDescriptor hcd = new HColumnDescriptor("value");
     htd.addFamily(hcd);
 
-    admin.createTable(htd, null);
+    admin.createTable(htd);
     try {
       RegionInfo hri = RegionInfoBuilder.newBuilder(tableName)
           .setStartKey(Bytes.toBytes("A"))
@@ -213,7 +213,7 @@ public class TestMaster {
     HColumnDescriptor hcd = new HColumnDescriptor("value");
     htd.addFamily(hcd);
 
-    admin.createTable(htd, null);
+    admin.createTable(htd);
     try {
       List<RegionInfo> tableRegions = admin.getRegions(tableName);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
index 1f0323e..355eb41 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
@@ -126,7 +126,7 @@ public class TestMasterMetricsWrapper {
       HTableDescriptor desc = new HTableDescriptor(table);
       byte[] FAMILY = Bytes.toBytes("FAMILY");
       desc.addFamily(new HColumnDescriptor(FAMILY));
-      TEST_UTIL.getHBaseAdmin().createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), 5);
+      TEST_UTIL.getAdmin().createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), 5);
 
       // wait till the table is assigned
       long timeoutTime = System.currentTimeMillis() + 1000;
@@ -148,7 +148,7 @@ public class TestMasterMetricsWrapper {
       assertEquals(5, regionNumberPair.getFirst().intValue());
       assertEquals(0, regionNumberPair.getSecond().intValue());
 
-      TEST_UTIL.getHBaseAdmin().offline(hri.getRegionName());
+      TEST_UTIL.getAdmin().offline(hri.getRegionName());
 
       timeoutTime = System.currentTimeMillis() + 800;
       RegionStates regionStates = master.getAssignmentManager().getRegionStates();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
index 7cf794a..182695c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
@@ -69,7 +69,7 @@ public class TestMergeTableRegionsWhileRSCrash {
   @BeforeClass
   public static void setupCluster() throws Exception {
     UTIL.startMiniCluster(1);
-    admin = UTIL.getHBaseAdmin();
+    admin = UTIL.getAdmin();
     byte[][] splitKeys = new byte[1][];
     splitKeys[0] = SPLITKEY;
     TABLE = UTIL.createTable(TABLE_NAME, CF, splitKeys);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java
index fe5d1a2..a55deb0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java
@@ -66,7 +66,7 @@ public class TestSplitRegionWhileRSCrash {
   @BeforeClass
   public static void setupCluster() throws Exception {
     UTIL.startMiniCluster(1);
-    admin = UTIL.getHBaseAdmin();
+    admin = UTIL.getAdmin();
     TABLE = UTIL.createTable(TABLE_NAME, CF);
     UTIL.waitTableAvailable(TABLE_NAME);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
index 839d611..5673ed8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
@@ -106,7 +106,7 @@ public class TestAssignmentOnRSCrash {
       throws Exception {
     final int NROWS = 100;
     int nkilled = 0;
-    for (RegionInfo hri: UTIL.getHBaseAdmin().getRegions(TEST_TABLE)) {
+    for (RegionInfo hri: UTIL.getAdmin().getRegions(TEST_TABLE)) {
       ServerName serverName = AssignmentTestingUtil.getServerHoldingRegion(UTIL, hri);
       if (AssignmentTestingUtil.isServerHoldingMeta(UTIL, serverName)) continue;
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java
index 1af9bd0..6fcdb39 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java
@@ -68,7 +68,7 @@ public class TestMasterAbortWhileMergingTable {
     UTIL.getConfiguration().set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
         MergeRegionObserver.class.getName());
     UTIL.startMiniCluster(3);
-    admin = UTIL.getHBaseAdmin();
+    admin = UTIL.getAdmin();
     byte[][] splitKeys = new byte[1][];
     splitKeys[0] = SPLITKEY;
     UTIL.createTable(TABLE_NAME, CF, splitKeys);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java
index 16ad373..0d77608 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java
@@ -67,7 +67,7 @@ public class TestModifyTableWhileMerging {
     //Set procedure executor thread to 1, making reproducing this issue of HBASE-20921 easier
     UTIL.getConfiguration().setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
     UTIL.startMiniCluster(1);
-    admin = UTIL.getHBaseAdmin();
+    admin = UTIL.getAdmin();
     byte[][] splitKeys = new byte[1][];
     splitKeys[0] = SPLITKEY;
     client = UTIL.createTable(TABLE_NAME, CF, splitKeys);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupCompactedFileOnRegionClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupCompactedFileOnRegionClose.java
index 6ae68f8..0bf4e78 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupCompactedFileOnRegionClose.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupCompactedFileOnRegionClose.java
@@ -24,13 +24,12 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.util.Collection;
-
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -40,7 +39,6 @@ import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -78,7 +76,7 @@ public class TestCleanupCompactedFileOnRegionClose {
     byte[] familyNameBytes = Bytes.toBytes(familyName);
     util.createTable(tableName, familyName);
 
-    HBaseAdmin hBaseAdmin = util.getHBaseAdmin();
+    Admin hBaseAdmin = util.getAdmin();
     Table table = util.getConnection().getTable(tableName);
 
     HRegionServer rs = util.getRSForFirstRegionInTable(tableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
index 8fa7f44..0ccda0d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
@@ -102,10 +102,9 @@ public class TestEndToEndSplitTransaction {
   }
 
 
-  /*
+  /**
    * This is the test for : HBASE-20940 This test will split the region and try to open an reference
    * over store file. Once store file has any reference, it makes sure that region can't be split
-   * @throws Exception
    */
   @Test
   public void testCanSplitJustAfterASplit() throws Exception {
@@ -125,7 +124,7 @@ public class TestEndToEndSplitTransaction {
       TEST_UTIL.loadTable(source, fam);
       List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(tableName);
       regions.get(0).forceSplit(null);
-      admin.split(tableName);
+      TEST_UTIL.getAsyncConnection().getAdmin().split(tableName);
 
       while (regions.size() <= 1) {
         regions = TEST_UTIL.getHBaseCluster().getRegions(tableName);
@@ -325,7 +324,7 @@ public class TestEndToEndSplitTransaction {
 
           Set<RegionInfo> regions = new TreeSet<>(RegionInfo.COMPARATOR);
           for (HRegionLocation loc : rl.getAllRegionLocations()) {
-            regions.add(loc.getRegionInfo());
+            regions.add(loc.getRegion());
           }
           verifyTableRegions(regions);
         }
@@ -504,7 +503,7 @@ public class TestEndToEndSplitTransaction {
     long start = System.currentTimeMillis();
     while (System.currentTimeMillis() - start < timeout) {
       HRegionLocation loc = MetaTableAccessor.getRegionLocation(conn, hri);
-      if (loc != null && !loc.getRegionInfo().isOffline()) {
+      if (loc != null && !loc.getRegion().isOffline()) {
         log("found region in META: " + hri.getRegionNameAsString());
         break;
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestNewVersionBehaviorFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestNewVersionBehaviorFromClientSide.java
index 3c3dadf..805decf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestNewVersionBehaviorFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestNewVersionBehaviorFromClientSide.java
@@ -82,7 +82,7 @@ public class TestNewVersionBehaviorFromClientSide {
     fam.setNewVersionBehavior(true);
     fam.setMaxVersions(3);
     table.addFamily(fam);
-    TEST_UTIL.getHBaseAdmin().createTable(table);
+    TEST_UTIL.getAdmin().createTable(table);
     return TEST_UTIL.getConnection().getTable(tableName);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
index 878ca75..2b8953e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
@@ -162,7 +162,7 @@ public class TestRegionServerAbort {
    */
   @Test
   public void testStopOverrideFromCoprocessor() throws Exception {
-    Admin admin = testUtil.getHBaseAdmin();
+    Admin admin = testUtil.getAdmin();
     HRegionServer regionserver = cluster.getRegionServer(0);
     admin.stopRegionServer(regionserver.getServerName().getHostAndPort());
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java
index 24329a0..387aa98 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -167,7 +167,7 @@ public class TestReplicator extends TestReplicationBase {
   }
 
   private void truncateTable(HBaseTestingUtility util, TableName tablename) throws IOException {
-    HBaseAdmin admin = util.getHBaseAdmin();
+    Admin admin = util.getAdmin();
     admin.disableTable(tableName);
     admin.truncateTable(tablename, false);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
index d39c0e6..f9ca754 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.protobuf.ServiceException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -74,7 +73,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 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.SnapshotProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
 
@@ -270,33 +268,6 @@ public final class SnapshotTestingUtils {
     }
   }
 
-  /**
-   * Helper method for testing async snapshot operations. Just waits for the
-   * given snapshot to complete on the server by repeatedly checking the master.
-   *
-   * @param master the master running the snapshot
-   * @param snapshot the snapshot to check
-   * @param sleep amount to sleep between checks to see if the snapshot is done
-   * @throws ServiceException if the snapshot fails
-   * @throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
-   */
-  public static void waitForSnapshotToComplete(HMaster master,
-      SnapshotProtos.SnapshotDescription snapshot, long sleep)
-          throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
-    final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder()
-        .setSnapshot(snapshot).build();
-    IsSnapshotDoneResponse done = IsSnapshotDoneResponse.newBuilder()
-        .buildPartial();
-    while (!done.getDone()) {
-      done = master.getMasterRpcServices().isSnapshotDone(null, request);
-      try {
-        Thread.sleep(sleep);
-      } catch (InterruptedException e) {
-        throw new org.apache.hbase.thirdparty.com.google.protobuf.ServiceException(e);
-      }
-    }
-  }
-
   /*
    * Take snapshot with maximum of numTries attempts, ignoring CorruptedSnapshotException
    * except for the last CorruptedSnapshotException
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
index e24d445..1d81dd7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
@@ -28,6 +28,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -57,7 +59,11 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
 
 /**
@@ -273,6 +279,38 @@ public class TestFlushSnapshotFromClient {
     }
   }
 
+  /**
+   * Helper method for testing async snapshot operations. Just waits for the given snapshot to
+   * complete on the server by repeatedly checking the master.
+   * @param master the master running the snapshot
+   * @param snapshot the snapshot to check
+   * @param sleep amount to sleep between checks to see if the snapshot is done
+   */
+  private static void waitForSnapshotToComplete(HMaster master,
+      SnapshotProtos.SnapshotDescription snapshot, long timeoutNanos) throws Exception {
+    final IsSnapshotDoneRequest request =
+      IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build();
+    long start = System.nanoTime();
+    while (System.nanoTime() - start < timeoutNanos) {
+      try {
+        IsSnapshotDoneResponse done = master.getMasterRpcServices().isSnapshotDone(null, request);
+        if (done.getDone()) {
+          return;
+        }
+      } catch (ServiceException e) {
+        // ignore UnknownSnapshotException, this is possible as for AsyncAdmin, the method will
+        // return immediately after sending out the request, no matter whether the master has
+        // processed the request or not.
+        if (!(e.getCause() instanceof UnknownSnapshotException)) {
+          throw e;
+        }
+      }
+
+      Thread.sleep(200);
+    }
+    throw new TimeoutException("Timeout waiting for snapshot " + snapshot + " to complete");
+  }
+
   @Test
   public void testAsyncFlushSnapshot() throws Exception {
     SnapshotProtos.SnapshotDescription snapshot = SnapshotProtos.SnapshotDescription.newBuilder()
@@ -285,7 +323,7 @@ public class TestFlushSnapshotFromClient {
 
     // constantly loop, looking for the snapshot to complete
     HMaster master = UTIL.getMiniHBaseCluster().getMaster();
-    SnapshotTestingUtils.waitForSnapshotToComplete(master, snapshot, 200);
+    waitForSnapshotToComplete(master, snapshot, TimeUnit.MINUTES.toNanos(1));
     LOG.info(" === Async Snapshot Completed ===");
     UTIL.getHBaseCluster().getMaster().getMasterFileSystem().logFileSystemState(LOG);
 
@@ -524,7 +562,6 @@ public class TestFlushSnapshotFromClient {
     SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TABLE_NAME);
   }
 
-
   protected void verifyRowCount(final HBaseTestingUtility util, final TableName tableName,
       long expectedRows) throws IOException {
     SnapshotTestingUtils.verifyRowCount(util, tableName, expectedRows);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFiles.java
index e85fc1a..5122464 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFiles.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFiles.java
@@ -335,7 +335,11 @@ public class TestBulkLoadHFiles {
 
     TableName tableName = htd.getTableName();
     if (!util.getAdmin().tableExists(tableName) && (preCreateTable || map != null)) {
-      util.getAdmin().createTable(htd, tableSplitKeys);
+      if (tableSplitKeys != null) {
+        util.getAdmin().createTable(htd, tableSplitKeys);
+      } else {
+        util.getAdmin().createTable(htd);
+      }
     }
 
     Configuration conf = util.getConfiguration();
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
index 565a9c7..2ebbb11 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
@@ -626,7 +626,11 @@ public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements TH
     try {
       TableDescriptor descriptor = tableDescriptorFromThrift(desc);
       byte[][] split = splitKeyFromThrift(splitKeys);
-      connectionCache.getAdmin().createTable(descriptor, split);
+      if (split != null) {
+        connectionCache.getAdmin().createTable(descriptor, split);
+      } else {
+        connectionCache.getAdmin().createTable(descriptor);
+      }
     } catch (IOException e) {
       throw getTIOError(e);
     }
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
index 1884fb0..d7aea33 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
@@ -720,6 +720,11 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
+  public List<RegionMetrics> getRegionMetrics(ServerName serverName) {
+    throw new NotImplementedException("getRegionMetrics not supported in ThriftAdmin");
+  }
+
+  @Override
   public List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName) {
     throw new NotImplementedException("getRegionMetrics not supported in ThriftAdmin");
   }
@@ -1127,8 +1132,8 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public List<UserPermission>
-      getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) {
+  public List<UserPermission> getUserPermissions(
+      GetUserPermissionsRequest getUserPermissionsRequest) {
     throw new NotImplementedException("getUserPermissions not supported in ThriftAdmin");
   }
 


[hbase] 08/13: HBASE-22297 Fix TestRegionMergeTransitionOnCluster and TestSplitTransactionOnCluster

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

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

commit 48ebece4fcbdd0bf584be9711c0af8cde43222fb
Author: zhangduo <zh...@apache.org>
AuthorDate: Tue Apr 23 22:22:39 2019 +0800

    HBASE-22297 Fix TestRegionMergeTransitionOnCluster and TestSplitTransactionOnCluster
    
    Signed-off-by: Michael Stack <st...@apache.org>
---
 .../TestRegionMergeTransactionOnCluster.java          | 19 +++++++++----------
 .../regionserver/TestSplitTransactionOnCluster.java   | 16 +++++++---------
 2 files changed, 16 insertions(+), 19 deletions(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
index ea93468..7b54ffb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
@@ -26,7 +26,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -63,6 +62,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
@@ -312,7 +312,6 @@ public class TestRegionMergeTransactionOnCluster {
     LOG.info("Starting " + name.getMethodName());
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final Admin admin = TEST_UTIL.getAdmin();
-    final int syncWaitTimeout = 10 * 60000; // 10min
 
     try {
       // Create table and load data.
@@ -326,8 +325,8 @@ public class TestRegionMergeTransactionOnCluster {
       am.offlineRegion(b);
       try {
         // Merge offline region. Region a is offline here
-        admin.mergeRegionsAsync(a.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), false)
-                .get(syncWaitTimeout, TimeUnit.MILLISECONDS);
+        FutureUtils.get(
+          admin.mergeRegionsAsync(a.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), false));
         fail("Offline regions should not be able to merge");
       } catch (DoNotRetryRegionException ie) {
         System.out.println(ie);
@@ -336,21 +335,21 @@ public class TestRegionMergeTransactionOnCluster {
 
       try {
         // Merge the same region: b and b.
-        admin.mergeRegionsAsync(b.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), true);
+        FutureUtils
+          .get(admin.mergeRegionsAsync(b.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), true));
         fail("A region should not be able to merge with itself, even forcifully");
       } catch (IOException ie) {
         assertTrue("Exception should mention regions not online",
-          StringUtils.stringifyException(ie).contains("region to itself")
-            && ie instanceof MergeRegionException);
+          StringUtils.stringifyException(ie).contains("region to itself") &&
+            ie instanceof MergeRegionException);
       }
 
       try {
         // Merge unknown regions
-        admin.mergeRegionsAsync(Bytes.toBytes("-f1"), Bytes.toBytes("-f2"), true);
+        FutureUtils.get(admin.mergeRegionsAsync(Bytes.toBytes("-f1"), Bytes.toBytes("-f2"), true));
         fail("Unknown region could not be merged");
       } catch (IOException ie) {
-        assertTrue("UnknownRegionException should be thrown",
-          ie instanceof UnknownRegionException);
+        assertTrue("UnknownRegionException should be thrown", ie instanceof UnknownRegionException);
       }
       table.close();
     } finally {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 405819e..d7b2f9e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.hadoop.conf.Configuration;
@@ -89,6 +90,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.RetryCounter;
@@ -328,7 +330,7 @@ public class TestSplitTransactionOnCluster {
       // We don't roll back here anymore. Instead we fail-fast on construction of the
       // split transaction. Catch the exception instead.
       try {
-        this.admin.splitRegionAsync(hri.getRegionName());
+        FutureUtils.get(this.admin.splitRegionAsync(hri.getRegionName()));
         fail();
       } catch (DoNotRetryRegionException e) {
         // Expected
@@ -510,17 +512,13 @@ public class TestSplitTransactionOnCluster {
   }
 
   /**
-   * Verifies HBASE-5806.  Here the case is that splitting is completed but before the
-   * CJ could remove the parent region the master is killed and restarted.
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws NodeExistsException
-   * @throws KeeperException
+   * Verifies HBASE-5806. Here the case is that splitting is completed but before the CJ could
+   * remove the parent region the master is killed and restarted.
    */
   @Test
   public void testMasterRestartAtRegionSplitPendingCatalogJanitor()
       throws IOException, InterruptedException, NodeExistsException,
-      KeeperException, ServiceException {
+      KeeperException, ServiceException, ExecutionException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
 
     // Create table then get the single region for our new table.
@@ -541,7 +539,7 @@ public class TestSplitTransactionOnCluster {
       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
       printOutRegions(server, "Initial regions: ");
       // Call split.
-      this.admin.splitRegionAsync(hri.getRegionName());
+      this.admin.splitRegionAsync(hri.getRegionName()).get();
       List<HRegion> daughters = checkAndGetDaughters(tableName);
 
       // Before cleanup, get a new master.


[hbase] 02/13: HBASE-22241 Fix TestRegionServerCoprocessorEndpoint

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

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

commit f405b9d645edf0a6c97f2a4a9326ad76ebc61797
Author: zhangduo <zh...@apache.org>
AuthorDate: Sun Apr 14 23:09:49 2019 +0800

    HBASE-22241 Fix TestRegionServerCoprocessorEndpoint
---
 .../hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)

diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java
index f180884..6d93ffc 100644
--- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java
+++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerE
 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyResponse;
 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyService;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
-import org.apache.hadoop.hbase.ipc.RemoteWithExtrasException;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -101,8 +100,7 @@ public class TestRegionServerCoprocessorEndpoint {
         DummyRegionServerEndpointProtos.DummyRequest.getDefaultInstance(), rpcCallback);
     assertEquals(null, rpcCallback.get());
     assertTrue(controller.failedOnException());
-    assertEquals(WHAT_TO_THROW.getClass().getName().trim(),
-        ((RemoteWithExtrasException) controller.getFailedOn().getCause()).getClassName().trim());
+    assertEquals(WHAT_TO_THROW.getClass(), controller.getFailedOn().getCause().getClass());
   }
 
   public static class DummyRegionServerEndpoint extends DummyService


[hbase] 03/13: HBASE-21725 Implement BufferedMutator Based on AsyncBufferedMutator

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

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

commit 82f0dbf6cd9636139e034c8bf861718ad0e46aae
Author: zhangduo <zh...@apache.org>
AuthorDate: Sun Apr 14 20:32:38 2019 +0800

    HBASE-21725 Implement BufferedMutator Based on AsyncBufferedMutator
---
 .../hadoop/hbase/client/BufferedMutator.java       |  10 ++
 .../BufferedMutatorOverAsyncBufferedMutator.java   | 175 +++++++++++++++++++++
 .../hadoop/hbase/client/BufferedMutatorParams.java |  23 ++-
 .../client/ConnectionOverAsyncConnection.java      |  19 ++-
 .../hadoop/hbase/client/TestBufferedMutator.java   |  82 ----------
 .../hadoop/hbase/client/TestBufferedMutator.java   |  90 +++++++++++
 6 files changed, 309 insertions(+), 90 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
index 7805f77..8ad6a79 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
@@ -62,7 +62,11 @@ import org.apache.yetus.audience.InterfaceAudience;
 public interface BufferedMutator extends Closeable {
   /**
    * Key to use setting non-default BufferedMutator implementation in Configuration.
+   * <p/>
+   * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it
+   *             any more.
    */
+  @Deprecated
   String CLASSNAME_KEY = "hbase.client.bufferedmutator.classname";
 
   /**
@@ -179,12 +183,18 @@ public interface BufferedMutator extends Closeable {
 
   /**
    * Set rpc timeout for this mutator instance
+   * @deprecated Since 3.0.0, will be removed in 4.0.0. Please set this through the
+   *             {@link BufferedMutatorParams}.
    */
+  @Deprecated
   void setRpcTimeout(int timeout);
 
   /**
    * Set operation timeout for this mutator instance
+   * @deprecated Since 3.0.0, will be removed in 4.0.0. Please set this through the
+   *             {@link BufferedMutatorParams}.
    */
+  @Deprecated
   void setOperationTimeout(int timeout);
 
   /**
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorOverAsyncBufferedMutator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorOverAsyncBufferedMutator.java
new file mode 100644
index 0000000..a7d4595
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorOverAsyncBufferedMutator.java
@@ -0,0 +1,175 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * {@link BufferedMutator} implementation based on {@link AsyncBufferedMutator}.
+ */
+@InterfaceAudience.Private
+class BufferedMutatorOverAsyncBufferedMutator implements BufferedMutator {
+
+  private final AsyncBufferedMutator mutator;
+
+  private final ExceptionListener listener;
+
+  private List<CompletableFuture<Void>> futures = new ArrayList<>();
+
+  private final ConcurrentLinkedQueue<Pair<Mutation, Throwable>> errors =
+    new ConcurrentLinkedQueue<>();
+
+  private final static int BUFFERED_FUTURES_THRESHOLD = 1024;
+
+  BufferedMutatorOverAsyncBufferedMutator(AsyncBufferedMutator mutator,
+      ExceptionListener listener) {
+    this.mutator = mutator;
+    this.listener = listener;
+  }
+
+  @Override
+  public TableName getName() {
+    return mutator.getName();
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return mutator.getConfiguration();
+  }
+
+  @Override
+  public void mutate(Mutation mutation) throws IOException {
+    mutate(Collections.singletonList(mutation));
+  }
+
+  private static final Pattern ADDR_MSG_MATCHER = Pattern.compile("Call to (\\S+) failed");
+
+  // not always work, so may return an empty string
+  private String getHostnameAndPort(Throwable error) {
+    Matcher matcher = ADDR_MSG_MATCHER.matcher(error.getMessage());
+    if (matcher.matches()) {
+      return matcher.group(1);
+    } else {
+      return "";
+    }
+  }
+
+  private RetriesExhaustedWithDetailsException makeError() {
+    List<Row> rows = new ArrayList<>();
+    List<Throwable> throwables = new ArrayList<>();
+    List<String> hostnameAndPorts = new ArrayList<>();
+    for (;;) {
+      Pair<Mutation, Throwable> pair = errors.poll();
+      if (pair == null) {
+        break;
+      }
+      rows.add(pair.getFirst());
+      throwables.add(pair.getSecond());
+      hostnameAndPorts.add(getHostnameAndPort(pair.getSecond()));
+    }
+    return new RetriesExhaustedWithDetailsException(throwables, rows, hostnameAndPorts);
+  }
+
+  @Override
+  public void mutate(List<? extends Mutation> mutations) throws IOException {
+    List<CompletableFuture<Void>> toBuffered = new ArrayList<>();
+    List<CompletableFuture<Void>> fs = mutator.mutate(mutations);
+    for (int i = 0, n = fs.size(); i < n; i++) {
+      CompletableFuture<Void> toComplete = new CompletableFuture<>();
+      final int index = i;
+      addListener(fs.get(index), (r, e) -> {
+        if (e != null) {
+          errors.add(Pair.newPair(mutations.get(index), e));
+          toComplete.completeExceptionally(e);
+        } else {
+          toComplete.complete(r);
+        }
+      });
+      toBuffered.add(toComplete);
+    }
+    synchronized (this) {
+      futures.addAll(toBuffered);
+      if (futures.size() > BUFFERED_FUTURES_THRESHOLD) {
+        tryCompleteFuture();
+      }
+      if (!errors.isEmpty()) {
+        RetriesExhaustedWithDetailsException error = makeError();
+        listener.onException(error, this);
+      }
+    }
+  }
+
+  private void tryCompleteFuture() {
+    futures = futures.stream().filter(f -> !f.isDone()).collect(Collectors.toList());
+  }
+
+  @Override
+  public void close() throws IOException {
+    flush();
+    mutator.close();
+  }
+
+  @Override
+  public void flush() throws IOException {
+    mutator.flush();
+    synchronized (this) {
+      List<CompletableFuture<Void>> toComplete = this.futures;
+      this.futures = new ArrayList<>();
+      try {
+        CompletableFuture.allOf(toComplete.toArray(new CompletableFuture<?>[toComplete.size()]))
+          .join();
+      } catch (CompletionException e) {
+        // just ignore, we will record the actual error in the errors field
+      }
+      if (!errors.isEmpty()) {
+        RetriesExhaustedWithDetailsException error = makeError();
+        listener.onException(error, this);
+      }
+    }
+  }
+
+  @Override
+  public long getWriteBufferSize() {
+    return mutator.getWriteBufferSize();
+  }
+
+  @Override
+  public void setRpcTimeout(int timeout) {
+    // no effect
+  }
+
+  @Override
+  public void setOperationTimeout(int timeout) {
+    // no effect
+  }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java
index 3f6c565..49fb77b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java
@@ -101,13 +101,21 @@ public class BufferedMutatorParams implements Cloneable {
     return this;
   }
 
+  /**
+   * @deprecated Since 3.0.0, will be removed in 4.0.0. We use a common timer in the whole client
+   *             implementation so you can not set it any more.
+   */
+  @Deprecated
   public long getWriteBufferPeriodicFlushTimerTickMs() {
     return writeBufferPeriodicFlushTimerTickMs;
   }
 
   /**
    * Set the TimerTick how often the buffer timeout if checked.
+   * @deprecated Since 3.0.0, will be removed in 4.0.0. We use a common timer in the whole client
+   *             implementation so you can not set it any more.
    */
+  @Deprecated
   public BufferedMutatorParams setWriteBufferPeriodicFlushTimerTickMs(long timerTickMs) {
     this.writeBufferPeriodicFlushTimerTickMs = timerTickMs;
     return this;
@@ -141,9 +149,12 @@ public class BufferedMutatorParams implements Cloneable {
   }
 
   /**
-   * @return Name of the class we will use when we construct a
-   * {@link BufferedMutator} instance or null if default implementation.
+   * @return Name of the class we will use when we construct a {@link BufferedMutator} instance or
+   *         null if default implementation.
+   * @deprecated Since 3.0.0, will be removed in 4.0.0. You can not set it any more as the
+   *             implementation has to use too many internal stuffs in HBase.
    */
+  @Deprecated
   public String getImplementationClassName() {
     return this.implementationClassName;
   }
@@ -151,7 +162,10 @@ public class BufferedMutatorParams implements Cloneable {
   /**
    * Specify a BufferedMutator implementation other than the default.
    * @param implementationClassName Name of the BufferedMutator implementation class
+   * @deprecated Since 3.0.0, will be removed in 4.0.0. You can not set it any more as the
+   *             implementation has to use too many internal stuffs in HBase.
    */
+  @Deprecated
   public BufferedMutatorParams implementationClassName(String implementationClassName) {
     this.implementationClassName = implementationClassName;
     return this;
@@ -169,11 +183,6 @@ public class BufferedMutatorParams implements Cloneable {
     return this;
   }
 
-  /*
-   * (non-Javadoc)
-   *
-   * @see java.lang.Object#clone()
-   */
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="CN_IDIOM_NO_SUPER_CALL",
     justification="The clone below is complete")
   @Override
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java
index dfe7d8f..8ec7ab8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java
@@ -87,7 +87,24 @@ class ConnectionOverAsyncConnection implements Connection {
 
   @Override
   public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException {
-    return oldConn.getBufferedMutator(params);
+    AsyncBufferedMutatorBuilder builder = conn.getBufferedMutatorBuilder(params.getTableName());
+    if (params.getRpcTimeout() != BufferedMutatorParams.UNSET) {
+      builder.setRpcTimeout(params.getRpcTimeout(), TimeUnit.MILLISECONDS);
+    }
+    if (params.getOperationTimeout() != BufferedMutatorParams.UNSET) {
+      builder.setOperationTimeout(params.getOperationTimeout(), TimeUnit.MILLISECONDS);
+    }
+    if (params.getWriteBufferSize() != BufferedMutatorParams.UNSET) {
+      builder.setWriteBufferSize(params.getWriteBufferSize());
+    }
+    if (params.getWriteBufferPeriodicFlushTimeoutMs() != BufferedMutatorParams.UNSET) {
+      builder.setWriteBufferPeriodicFlush(params.getWriteBufferPeriodicFlushTimeoutMs(),
+        TimeUnit.MILLISECONDS);
+    }
+    if (params.getMaxKeyValueSize() != BufferedMutatorParams.UNSET) {
+      builder.setMaxKeyValueSize(params.getMaxKeyValueSize());
+    }
+    return new BufferedMutatorOverAsyncBufferedMutator(builder.build(), params.getListener());
   }
 
   @Override
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java
deleted file mode 100644
index 96bb846..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java
+++ /dev/null
@@ -1,82 +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.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-
-@Category({ SmallTests.class, ClientTests.class })
-public class TestBufferedMutator {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestBufferedMutator.class);
-
-  @Rule
-  public TestName name = new TestName();
-
-  /**
-   * My BufferedMutator. Just to prove that I can insert a BM other than default.
-   */
-  public static class MyBufferedMutator extends BufferedMutatorImpl {
-    MyBufferedMutator(ConnectionImplementation conn, RpcRetryingCallerFactory rpcCallerFactory,
-        RpcControllerFactory rpcFactory, BufferedMutatorParams params) {
-      super(conn, rpcCallerFactory, rpcFactory, params);
-    }
-  }
-
-  @Test
-  public void testAlternateBufferedMutatorImpl() throws IOException {
-    BufferedMutatorParams params =
-      new BufferedMutatorParams(TableName.valueOf(name.getMethodName()));
-    Configuration conf = HBaseConfiguration.create();
-    conf.set(AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY, DoNothingAsyncRegistry.class.getName());
-    try (ConnectionImplementation connection = ConnectionFactory.createConnectionImpl(conf, null,
-      UserProvider.instantiate(conf).getCurrent())) {
-      BufferedMutator bm = connection.getBufferedMutator(params);
-      // Assert we get default BM if nothing specified.
-      assertTrue(bm instanceof BufferedMutatorImpl);
-      // Now try and set my own BM implementation.
-      params.implementationClassName(MyBufferedMutator.class.getName());
-      bm = connection.getBufferedMutator(params);
-      assertTrue(bm instanceof MyBufferedMutator);
-    }
-    // Now try creating a Connection after setting an alterate BufferedMutator into
-    // the configuration and confirm we get what was expected.
-    conf.set(BufferedMutator.CLASSNAME_KEY, MyBufferedMutator.class.getName());
-    try (Connection connection = ConnectionFactory.createConnectionImpl(conf, null,
-      UserProvider.instantiate(conf).getCurrent())) {
-      BufferedMutator bm = connection.getBufferedMutator(params);
-      assertTrue(bm instanceof MyBufferedMutator);
-    }
-  }
-}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java
new file mode 100644
index 0000000..23e69ee
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertArrayEquals;
+
+import java.io.IOException;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestBufferedMutator {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestBufferedMutator.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static TableName TABLE_NAME = TableName.valueOf("test");
+
+  private static byte[] CF = Bytes.toBytes("cf");
+
+  private static byte[] CQ = Bytes.toBytes("cq");
+
+  private static int COUNT = 1024;
+
+  private static byte[] VALUE = new byte[1024];
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.startMiniCluster(1);
+    TEST_UTIL.createTable(TABLE_NAME, CF);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws Exception {
+    try (BufferedMutator mutator = TEST_UTIL.getConnection().getBufferedMutator(TABLE_NAME)) {
+      mutator.mutate(IntStream.range(0, COUNT / 2)
+        .mapToObj(i -> new Put(Bytes.toBytes(i)).addColumn(CF, CQ, VALUE))
+        .collect(Collectors.toList()));
+      mutator.flush();
+      mutator.mutate(IntStream.range(COUNT / 2, COUNT)
+        .mapToObj(i -> new Put(Bytes.toBytes(i)).addColumn(CF, CQ, VALUE))
+        .collect(Collectors.toList()));
+      mutator.close();
+      verifyData();
+    }
+  }
+
+  private void verifyData() throws IOException {
+    try (Table table = TEST_UTIL.getConnection().getTable(TABLE_NAME)) {
+      for (int i = 0; i < COUNT; i++) {
+        Result r = table.get(new Get(Bytes.toBytes(i)));
+        assertArrayEquals(VALUE, ((Result) r).getValue(CF, CQ));
+      }
+    }
+  }
+}


[hbase] 06/13: HBASE-22281 Fix failed shell UTs

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

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

commit e5fe6ae125a4a7a49605cb89de6c4cc0e69956b2
Author: zhangduo <zh...@apache.org>
AuthorDate: Mon Apr 22 22:00:43 2019 +0800

    HBASE-22281 Fix failed shell UTs
---
 hbase-shell/src/main/ruby/hbase/admin.rb | 44 ++++++++++++++++++++++++--------
 hbase-shell/src/main/ruby/hbase/hbase.rb | 23 +++++++++++------
 hbase-shell/src/main/ruby/hbase/table.rb |  2 +-
 3 files changed, 50 insertions(+), 19 deletions(-)

diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index a9f69b5..13f4eb0 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -54,7 +54,7 @@ module Hbase
     # Requests a table or region or region server flush
     def flush(name)
       @admin.flushRegion(name.to_java_bytes)
-    rescue java.lang.IllegalArgumentException
+    rescue java.lang.IllegalArgumentException, org.apache.hadoop.hbase.UnknownRegionException
       # Unknown region. Try table.
       begin
         @admin.flush(TableName.valueOf(name))
@@ -79,9 +79,17 @@ module Hbase
       end
 
       begin
-        @admin.compactRegion(table_or_region_name.to_java_bytes, family_bytes)
-      rescue java.lang.IllegalArgumentException => e
-        @admin.compact(TableName.valueOf(table_or_region_name), family_bytes, compact_type)
+        if family_bytes.nil?
+          @admin.compactRegion(table_or_region_name.to_java_bytes)
+        else
+          @admin.compactRegion(table_or_region_name.to_java_bytes, family_bytes)
+        end
+      rescue java.lang.IllegalArgumentException, org.apache.hadoop.hbase.UnknownRegionException
+        if family_bytes.nil?
+          @admin.compact(TableName.valueOf(table_or_region_name), compact_type)
+        else
+          @admin.compact(TableName.valueOf(table_or_region_name), family_bytes, compact_type)
+        end
       end
     end
 
@@ -124,9 +132,17 @@ module Hbase
       end
 
       begin
-        @admin.majorCompactRegion(table_or_region_name.to_java_bytes, family_bytes)
-      rescue java.lang.IllegalArgumentException => e
-        @admin.majorCompact(TableName.valueOf(table_or_region_name), family_bytes, compact_type)
+        if family_bytes.nil?
+          @admin.majorCompactRegion(table_or_region_name.to_java_bytes)
+        else
+          @admin.majorCompactRegion(table_or_region_name.to_java_bytes, family_bytes)
+        end
+      rescue java.lang.IllegalArgumentException, org.apache.hadoop.hbase.UnknownRegionException
+        if family_bytes.nil?
+          @admin.majorCompact(TableName.valueOf(table_or_region_name), compact_type)
+        else
+          @admin.majorCompact(TableName.valueOf(table_or_region_name), family_bytes, compact_type)
+        end
       end
     end
 
@@ -144,9 +160,17 @@ module Hbase
       split_point_bytes = nil
       split_point_bytes = split_point.to_java_bytes unless split_point.nil?
       begin
-        @admin.splitRegionAsync(table_or_region_name.to_java_bytes, split_point_bytes).get
-      rescue java.lang.IllegalArgumentException, org.apache.hadoop.hbase.UnknownRegionException => e
-        @admin.split(TableName.valueOf(table_or_region_name), split_point_bytes)
+        if split_point_bytes.nil?
+          org.apache.hadoop.hbase.util.FutureUtils.get(@admin.splitRegionAsync(table_or_region_name.to_java_bytes))
+        else
+          org.apache.hadoop.hbase.util.FutureUtils.get(@admin.splitRegionAsync(table_or_region_name.to_java_bytes, split_point_bytes))
+        end
+      rescue java.lang.IllegalArgumentException, org.apache.hadoop.hbase.UnknownRegionException
+        if split_point_bytes.nil?
+          @admin.split(TableName.valueOf(table_or_region_name))
+        else
+          @admin.split(TableName.valueOf(table_or_region_name), split_point_bytes)
+        end
       end
     end
 
diff --git a/hbase-shell/src/main/ruby/hbase/hbase.rb b/hbase-shell/src/main/ruby/hbase/hbase.rb
index 1f37f99..a9b35ed 100644
--- a/hbase-shell/src/main/ruby/hbase/hbase.rb
+++ b/hbase-shell/src/main/ruby/hbase/hbase.rb
@@ -42,16 +42,21 @@ module Hbase
         configuration.setInt('hbase.client.retries.number', 7)
         configuration.setInt('hbase.ipc.client.connect.max.retries', 3)
       end
-      @connection = ConnectionFactory.createConnection(configuration)
     end
 
+    def connection
+      if @connection.nil?
+        @connection = ConnectionFactory.createConnection(configuration)
+      end
+      @connection
+    end
     # Returns ruby's Admin class from admin.rb
     def admin
-      ::Hbase::Admin.new(@connection)
+      ::Hbase::Admin.new(self.connection)
     end
 
     def rsgroup_admin
-      ::Hbase::RSGroupAdmin.new(@connection)
+      ::Hbase::RSGroupAdmin.new(self.connection)
     end
 
     def taskmonitor
@@ -60,7 +65,7 @@ module Hbase
 
     # Create new one each time
     def table(table, shell)
-      ::Hbase::Table.new(@connection.getTable(TableName.valueOf(table)), shell)
+      ::Hbase::Table.new(self.connection.getTable(TableName.valueOf(table)), shell)
     end
 
     def replication_admin
@@ -68,19 +73,21 @@ module Hbase
     end
 
     def security_admin
-      ::Hbase::SecurityAdmin.new(@connection.getAdmin)
+      ::Hbase::SecurityAdmin.new(self.connection.getAdmin)
     end
 
     def visibility_labels_admin
-      ::Hbase::VisibilityLabelsAdmin.new(@connection.getAdmin)
+      ::Hbase::VisibilityLabelsAdmin.new(self.connection.getAdmin)
     end
 
     def quotas_admin
-      ::Hbase::QuotasAdmin.new(@connection.getAdmin)
+      ::Hbase::QuotasAdmin.new(self.connection.getAdmin)
     end
 
     def shutdown
-      @connection.close
+      if @connection != nil
+        @connection.close
+      end
     end
   end
 end
diff --git a/hbase-shell/src/main/ruby/hbase/table.rb b/hbase-shell/src/main/ruby/hbase/table.rb
index 8c7144d..53b090e 100644
--- a/hbase-shell/src/main/ruby/hbase/table.rb
+++ b/hbase-shell/src/main/ruby/hbase/table.rb
@@ -717,7 +717,7 @@ EOF
 
     # Returns a list of column names in the table
     def get_all_columns
-      @table.table_descriptor.getFamilies.map do |family|
+      @table.descriptor.getColumnFamilies.map do |family|
         "#{family.getNameAsString}:"
       end
     end