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

[hbase] branch master updated: HBASE-22002 Remove the deprecated methods in Admin interface

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 3c5761e  HBASE-22002 Remove the deprecated methods in Admin interface
3c5761e is described below

commit 3c5761e3656c2cddd59b069d332c4a3e13c87a56
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Wed Mar 13 17:21:43 2019 +0800

    HBASE-22002 Remove the deprecated methods in Admin interface
    
    Signed-off-by: Zheng Hu <op...@gmail.com>
---
 .../archetypes/exemplars/client/HelloHBase.java    |   5 +-
 .../exemplars/shaded_client/HelloHBase.java        |   5 +-
 .../hadoop/hbase/backup/impl/BackupManager.java    |  12 +-
 .../hadoop/hbase/backup/TestIncrementalBackup.java |   5 +-
 .../java/org/apache/hadoop/hbase/client/Admin.java | 961 +--------------------
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java | 499 +----------
 .../org/apache/hadoop/hbase/client/HTable.java     |  66 +-
 .../hbase/security/access/AccessControlClient.java |  15 +-
 .../org/apache/hadoop/hbase/util/FutureUtils.java  |   2 +-
 .../coprocessor/TestCoprocessorTableEndpoint.java  |   2 +-
 .../hadoop/hbase/DistributedHBaseCluster.java      |   6 +-
 .../hbase/IntegrationTestDDLMasterFailover.java    |  13 +-
 .../hadoop/hbase/IntegrationTestIngestWithMOB.java |   2 +-
 .../hbase/IntegrationTestRegionReplicaPerf.java    |   2 +-
 .../apache/hadoop/hbase/chaos/actions/Action.java  |   2 +-
 .../actions/CompactRandomRegionOfTableAction.java  |   9 +-
 .../actions/FlushRandomRegionOfTableAction.java    |   9 +-
 .../MergeRandomAdjacentRegionsOfTableAction.java   |   9 +-
 .../chaos/actions/MoveRegionsOfTableAction.java    |   6 +-
 .../actions/SplitRandomRegionOfTableAction.java    |  11 +-
 .../hbase/mapreduce/IntegrationTestBulkLoad.java   |   4 +-
 .../hadoop/hbase/mttr/IntegrationTestMTTR.java     |   7 +-
 .../hbase/rsgroup/IntegrationTestRSGroup.java      |   2 +-
 .../apache/hadoop/hbase/PerformanceEvaluation.java |   2 +-
 .../hbase/mapreduce/TestHFileOutputFormat2.java    |  14 +-
 .../hadoop/hbase/snapshot/TestExportSnapshot.java  |  34 +-
 .../snapshot/TestExportSnapshotNoCluster.java      |   3 +-
 .../hbase/rest/NamespacesInstanceResource.java     |  19 +-
 .../apache/hadoop/hbase/rest/SchemaResource.java   |   4 +-
 .../hadoop/hbase/rest/PerformanceEvaluation.java   |   4 +-
 .../hadoop/hbase/coprocessor/BulkLoadObserver.java |   4 +-
 .../apache/hadoop/hbase/master/CatalogJanitor.java |   6 +-
 .../hbase/master/assignment/RegionStateStore.java  |   8 +-
 .../master/balancer/RegionLocationFinder.java      |   4 +-
 .../master/normalizer/SplitNormalizationPlan.java  |   6 +-
 .../hadoop/hbase/mob/ExpiredMobFileCleaner.java    |   6 +-
 .../hadoop/hbase/regionserver/CompactSplit.java    |   4 +-
 .../DelimitedKeyPrefixRegionSplitPolicy.java       |   4 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java    |   2 +-
 .../java/org/apache/hadoop/hbase/tool/Canary.java  |  10 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java    |  12 +-
 .../resources/hbase-webapps/master/rsgroup.jsp     |   6 +-
 .../main/resources/hbase-webapps/master/table.jsp  |   2 +-
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |  29 +-
 .../hadoop/hbase/TestClientClusterStatus.java      |  24 +-
 .../org/apache/hadoop/hbase/TestNamespace.java     |  36 +-
 .../hbase/TestPartialResultsFromClientSide.java    |   2 +-
 .../org/apache/hadoop/hbase/TestRegionLoad.java    |  17 +-
 .../CloneSnapshotFromClientNormalTestBase.java     |   2 +-
 .../client/CloneSnapshotFromClientTestBase.java    |  16 +-
 .../RestoreSnapshotFromClientSimpleTestBase.java   |   3 +-
 .../client/RestoreSnapshotFromClientTestBase.java  |  16 +-
 .../org/apache/hadoop/hbase/client/TestAdmin1.java | 292 +++----
 .../org/apache/hadoop/hbase/client/TestAdmin2.java | 140 ++-
 .../hbase/client/TestAlwaysSetScannerId.java       |   5 +-
 .../hadoop/hbase/client/TestClientTimeouts.java    |   2 +-
 .../hbase/client/TestConnectionImplementation.java |  16 +-
 .../hadoop/hbase/client/TestFromClientSide.java    |  16 +-
 .../hadoop/hbase/client/TestFromClientSide3.java   |  30 +-
 .../hbase/client/TestHBaseAdminNoCluster.java      |  20 +-
 .../hadoop/hbase/client/TestLeaseRenewal.java      |   9 +-
 .../hadoop/hbase/client/TestMetaWithReplicas.java  |   2 +-
 ...oneSnapshotFromClientCloneLinksAfterDelete.java |   5 +-
 .../hbase/client/TestMultiRespectsLimits.java      |   2 +-
 .../hbase/client/TestReplicaWithCluster.java       |   8 +-
 .../hbase/client/TestScanWithoutFetchingData.java  |   5 +-
 .../client/TestSnapshotCloneIndependence.java      |  20 +-
 .../hbase/client/TestSnapshotFromClient.java       |  43 +-
 .../hadoop/hbase/client/TestSnapshotMetadata.java  |  21 +-
 .../client/TestSnapshotTemporaryDirectory.java     |  51 +-
 .../hbase/client/TestSplitOrMergeStatus.java       |  64 +-
 .../hadoop/hbase/client/TestTableFavoredNodes.java |  28 +-
 .../hbase/client/TestTableSnapshotScanner.java     |   3 +-
 .../TestReplicationAdminWithClusters.java          |  26 +-
 .../hbase/coprocessor/TestCoprocessorMetrics.java  |   2 +-
 .../hbase/coprocessor/TestMasterObserver.java      |  10 +-
 .../io/encoding/TestLoadAndSwitchEncodeOnDisk.java |   4 +-
 .../master/TestCatalogJanitorInMemoryStates.java   |  21 +-
 .../hbase/master/TestMasterBalanceThrottling.java  |   4 +-
 .../TestMasterOperationsForRegionReplicas.java     |   6 +-
 .../hbase/master/TestMasterStatusServlet.java      |  10 +-
 .../master/assignment/TestAssignmentOnRSCrash.java |   2 +-
 .../master/assignment/TestRogueRSAssignment.java   |  16 +-
 .../balancer/TestFavoredNodeTableImport.java       |   9 +-
 .../TestFavoredStochasticBalancerPickers.java      |   2 +-
 .../TestFavoredStochasticLoadBalancer.java         |  50 +-
 .../hbase/master/locking/TestLockManager.java      |   8 +-
 .../TestSimpleRegionNormalizerOnCluster.java       |   8 +-
 .../procedure/TestCloneSnapshotProcedure.java      |   2 +-
 .../TestDeleteColumnFamilyProcedureFromClient.java |   6 +-
 .../TestMasterFailoverWithProcedures.java          |   2 +-
 .../master/procedure/TestModifyTableProcedure.java |  34 +-
 .../hbase/master/procedure/TestProcedureAdmin.java |   4 +-
 .../procedure/TestRestoreSnapshotProcedure.java    |   9 +-
 .../procedure/TestTableDDLProcedureBase.java       |   8 +-
 .../TestTableDescriptorModificationFromClient.java |  10 +-
 .../hbase/mob/compactions/TestMobCompactor.java    |  11 +-
 .../hbase/namespace/TestNamespaceAuditor.java      |  10 +-
 .../hbase/procedure/TestProcedureManager.java      |   2 +-
 .../hadoop/hbase/quotas/TestTablesWithQuotas.java  |   2 +-
 .../regionserver/TestCompactionFileNotFound.java   |   9 +-
 .../regionserver/TestEndToEndSplitTransaction.java |   3 +-
 .../TestRegionMergeTransactionOnCluster.java       |   6 +-
 .../hadoop/hbase/regionserver/TestRegionOpen.java  |   4 +-
 .../regionserver/TestRegionServerMetrics.java      |  14 -
 .../TestRegionReplicaReplicationEndpoint.java      |   6 +-
 .../security/access/TestAccessController.java      |  10 +-
 .../TestCoprocessorWhitelistMasterObserver.java    |  11 +-
 .../hbase/security/access/TestRpcAccessChecks.java |   2 +-
 .../security/visibility/TestVisibilityLabels.java  |   2 +-
 .../hbase/snapshot/SnapshotTestingUtils.java       |   2 +-
 .../snapshot/TestFlushSnapshotFromClient.java      |  42 +-
 .../TestRestoreFlushSnapshotFromClient.java        |  20 +-
 .../apache/hadoop/hbase/util/TestHBaseFsckMOB.java |   2 +-
 .../hbase/util/TestMiniClusterLoadSequential.java  |   5 +-
 .../util/hbck/OfflineMetaRebuildTestCore.java      |  12 +-
 hbase-shell/src/main/ruby/hbase/admin.rb           |  91 +-
 hbase-shell/src/main/ruby/hbase/quotas.rb          |  33 +-
 hbase-shell/src/main/ruby/hbase/security.rb        |   4 +-
 hbase-shell/src/main/ruby/hbase/taskmonitor.rb     |   2 +-
 .../hbase/thrift2/ThriftHBaseServiceHandler.java   |  10 +-
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   | 251 +-----
 122 files changed, 904 insertions(+), 2650 deletions(-)

diff --git a/hbase-archetypes/hbase-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/client/HelloHBase.java b/hbase-archetypes/hbase-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/client/HelloHBase.java
index 5164ab2..a9e522f 100644
--- a/hbase-archetypes/hbase-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/client/HelloHBase.java
+++ b/hbase-archetypes/hbase-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/client/HelloHBase.java
@@ -66,9 +66,8 @@ public final class HelloHBase {
      * CLASSPATH, to enable creation of Connection to HBase via ZooKeeper.
      */
     try (Connection connection = ConnectionFactory.createConnection();
-            Admin admin = connection.getAdmin()) {
-
-      admin.getClusterStatus(); // assure connection successfully established
+        Admin admin = connection.getAdmin()) {
+      admin.getClusterMetrics(); // assure connection successfully established
       System.out.println("\n*** Hello HBase! -- Connection has been "
               + "established via ZooKeeper!!\n");
 
diff --git a/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
index 94a1e71..94bdf69 100644
--- a/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
+++ b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
@@ -65,9 +65,8 @@ public final class HelloHBase {
      * CLASSPATH, to enable creation of Connection to HBase via ZooKeeper.
      */
     try (Connection connection = ConnectionFactory.createConnection();
-            Admin admin = connection.getAdmin()) {
-
-      admin.getClusterStatus(); // assure connection successfully established
+        Admin admin = connection.getAdmin()) {
+      admin.getClusterMetrics(); // assure connection successfully established
       System.out.println("\n*** Hello HBase! -- Connection has been "
               + "established via ZooKeeper!!\n");
 
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
index 90677fe..d49aef2 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
@@ -26,11 +26,9 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupHFileCleaner;
 import org.apache.hadoop.hbase.backup.BackupInfo;
@@ -45,15 +43,17 @@ import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
 import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.procedure.ProcedureManagerHost;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 /**
  * Handles backup requests, creates backup info records in backup system table to keep track of
  * backup sessions, dispatches backup request.
@@ -208,9 +208,9 @@ public class BackupManager implements Closeable {
     if (type == BackupType.FULL && (tableList == null || tableList.isEmpty())) {
       // If table list is null for full backup, which means backup all tables. Then fill the table
       // list with all user tables from meta. It no table available, throw the request exception.
-      HTableDescriptor[] htds = null;
+      List<TableDescriptor> htds = null;
       try (Admin admin = conn.getAdmin()) {
-        htds = admin.listTables();
+        htds = admin.listTableDescriptors();
       } catch (Exception e) {
         throw new BackupException(e);
       }
@@ -219,7 +219,7 @@ public class BackupManager implements Closeable {
         throw new BackupException("No table exists for full backup of all tables.");
       } else {
         tableList = new ArrayList<>();
-        for (HTableDescriptor hTableDescriptor : htds) {
+        for (TableDescriptor hTableDescriptor : htds) {
           TableName tn = hTableDescriptor.getTableName();
           if (tn.equals(BackupSystemTable.getTableName(conf))) {
             // skip backup system table
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 6e15238..525845c 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
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.backup;
 
 import static org.junit.Assert.assertTrue;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -134,8 +133,8 @@ public class TestIncrementalBackup extends TestBackupBase {
       long startSplitTime = EnvironmentEdgeManager.currentTime();
 
       try {
-        admin.splitRegion(name);
-      } catch (IOException e) {
+        admin.splitRegionAsync(name, null).get();
+      } catch (Exception e) {
         // although split fail, this may not affect following check in current API,
         // exception will be thrown.
         LOG.debug("region is not splittable, because " + e);
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 a0c5401..b65b1c4 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
@@ -34,9 +34,6 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.RegionMetrics;
@@ -49,7 +46,6 @@ 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.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
@@ -62,8 +58,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.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -122,18 +116,6 @@ public interface Admin extends Abortable, Closeable {
   /**
    * List all the userspace tables.
    *
-   * @return an array of read-only HTableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #listTableDescriptors()}.
-   * @see #listTableDescriptors()
-   */
-  @Deprecated
-  HTableDescriptor[] listTables() throws IOException;
-
-  /**
-   * List all the userspace tables.
-   *
    * @return a list of TableDescriptors
    * @throws IOException if a remote or network exception occurs
    */
@@ -143,85 +125,27 @@ public interface Admin extends Abortable, Closeable {
    * List all the userspace tables that match the given pattern.
    *
    * @param pattern The compiled regular expression to match against
-   * @return an array of read-only HTableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @see #listTables()
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #listTableDescriptors(java.util.regex.Pattern)}.
-   * @see #listTableDescriptors(Pattern)
-   */
-  @Deprecated
-  HTableDescriptor[] listTables(Pattern pattern) throws IOException;
-
-  /**
-   * List all the userspace tables that match the given pattern.
-   *
-   * @param pattern The compiled regular expression to match against
    * @return a list of TableDescriptors
    * @throws IOException if a remote or network exception occurs
-   * @see #listTables()
+   * @see #listTableDescriptors()
    */
   default List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException {
     return listTableDescriptors(pattern, false);
   }
 
   /**
-   * List all the userspace tables matching the given regular expression.
-   *
-   * @param regex The regular expression to match against
-   * @return a list of read-only HTableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @see #listTableDescriptors(Pattern)
-   * @deprecated since 2.0 version and will be removed in 3.0 version. Use
-   *             {@link #listTableDescriptors(Pattern)} instead.
-   */
-  @Deprecated
-  HTableDescriptor[] listTables(String regex) throws IOException;
-
-  /**
-   * List all the tables matching the given pattern.
-   *
-   * @param pattern The compiled regular expression to match against
-   * @param includeSysTables <code>false</code> to match only against userspace tables
-   * @return an array of read-only HTableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @see #listTables()
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #listTableDescriptors(java.util.regex.Pattern, boolean)}.
-   * @see #listTableDescriptors(java.util.regex.Pattern, boolean)
-   */
-  @Deprecated
-  HTableDescriptor[] listTables(Pattern pattern, boolean includeSysTables)
-      throws IOException;
-
-  /**
    * List all the tables matching the given pattern.
    *
    * @param pattern The compiled regular expression to match against
    * @param includeSysTables <code>false</code> to match only against userspace tables
    * @return a list of TableDescriptors
    * @throws IOException if a remote or network exception occurs
-   * @see #listTables()
+   * @see #listTableDescriptors()
    */
   List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
       throws IOException;
 
   /**
-   * List all the tables matching the given pattern.
-   *
-   * @param regex The regular expression to match against
-   * @param includeSysTables <code>false</code> to match only against userspace tables
-   * @return an array of read-only HTableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @see #listTables(java.util.regex.Pattern, boolean)
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #listTableDescriptors(Pattern, boolean)}.
-   */
-  @Deprecated
-  HTableDescriptor[] listTables(String regex, boolean includeSysTables)
-      throws IOException;
-
-  /**
    * List all of the names of userspace tables.
    *
    * @return TableName[] table names
@@ -241,17 +165,6 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * List all of the names of userspace tables.
-   * @param regex The regular expression to match against
-   * @return TableName[] table names
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version. Use
-   *             {@link #listTableNames(Pattern)} instead.
-   */
-  @Deprecated
-  TableName[] listTableNames(String regex) throws IOException;
-
-  /**
-   * List all of the names of userspace tables.
    * @param pattern The regular expression to match against
    * @param includeSysTables <code>false</code> to match only against userspace tables
    * @return TableName[] table names
@@ -261,33 +174,6 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
-   * List all of the names of userspace tables.
-   * @param regex The regular expression to match against
-   * @param includeSysTables <code>false</code> to match only against userspace tables
-   * @return TableName[] table names
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version. Use
-   *             {@link #listTableNames(Pattern, boolean)} instead.
-   */
-  @Deprecated
-  TableName[] listTableNames(String regex, boolean includeSysTables)
-      throws IOException;
-
-  /**
-   * Get a table descriptor.
-   *
-   * @param tableName as a {@link TableName}
-   * @return the read-only tableDescriptor
-   * @throws org.apache.hadoop.hbase.TableNotFoundException
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #getDescriptor(TableName)}.
-   */
-  @Deprecated
-  HTableDescriptor getTableDescriptor(TableName tableName)
-      throws TableNotFoundException, IOException;
-
-  /**
    * Get a table descriptor.
    *
    * @param tableName as a {@link TableName}
@@ -390,47 +276,7 @@ public interface Admin extends Abortable, Closeable {
   Future<Void> deleteTableAsync(TableName tableName) throws IOException;
 
   /**
-   * Deletes tables matching the passed in pattern and wait on completion. Warning: Use this method
-   * carefully, there is no prompting and the effect is immediate. Consider using {@link
-   * #listTableDescriptors(Pattern)}
-   * and {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
-   *
-   * @param regex The regular expression to match table names against
-   * @return Table descriptors for tables that couldn't be deleted.
-   *         The return htds are read-only
-   * @throws IOException
-   * @see #deleteTables(java.util.regex.Pattern)
-   * @see #deleteTable(org.apache.hadoop.hbase.TableName)
-   * @deprecated since 2.0 version and will be removed in 3.0 version
-   *             This is just a trivial helper method without any magic.
-   *             Consider using {@link #listTableDescriptors(Pattern)}
-   *             and {@link #deleteTable(TableName)}
-   */
-  @Deprecated
-  HTableDescriptor[] deleteTables(String regex) throws IOException;
-
-  /**
-   * Delete tables matching the passed in pattern and wait on completion. Warning: Use this method
-   * carefully, there is no prompting and the effect is immediate. Consider using {@link
-   * #listTableDescriptors(java.util.regex.Pattern)} and
-   * {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
-   *
-   * @param pattern The pattern to match table names against
-   * @return Table descriptors for tables that couldn't be deleted
-   *         The return htds are read-only
-   * @throws IOException
-   * @deprecated since 2.0 version and will be removed in 3.0 version
-   *             This is just a trivial helper method without any magic.
-   *             Consider using {@link #listTableDescriptors(java.util.regex.Pattern)}
-   *             and {@link #deleteTable(TableName)}
-   */
-  @Deprecated
-  HTableDescriptor[] deleteTables(Pattern pattern) throws IOException;
-
-  /**
-   * Truncate a table.
-   * Synchronous operation.
-   *
+   * Truncate a table. Synchronous operation.
    * @param tableName name of table to truncate
    * @param preserveSplits <code>true</code> if the splits should be preserved
    * @throws IOException if a remote or network exception occurs
@@ -484,43 +330,6 @@ public interface Admin extends Abortable, Closeable {
   Future<Void> enableTableAsync(TableName tableName) throws IOException;
 
   /**
-   * Enable tables matching the passed in pattern and wait on completion. Warning: Use this method
-   * carefully, there is no prompting and the effect is immediate. Consider using {@link
-   * #listTableDescriptors(Pattern)} and {@link #enableTable(org.apache.hadoop.hbase.TableName)}
-   *
-   * @param regex The regular expression to match table names against
-   * @throws IOException
-   * @return Table descriptors for tables that couldn't be enabled.
-   *         The return HTDs are read-only.
-   * @see #enableTables(java.util.regex.Pattern)
-   * @see #enableTable(org.apache.hadoop.hbase.TableName)
-   * @deprecated since 2.0 version and will be removed in 3.0 version
-   *             This is just a trivial helper method without any magic.
-   *             Consider using {@link #listTableDescriptors(Pattern)}
-   *             and {@link #enableTable(org.apache.hadoop.hbase.TableName)}
-   */
-  @Deprecated
-  HTableDescriptor[] enableTables(String regex) throws IOException;
-
-  /**
-   * Enable tables matching the passed in pattern and wait on completion. Warning: Use this method
-   * carefully, there is no prompting and the effect is immediate. Consider using {@link
-   * #listTableDescriptors(java.util.regex.Pattern)} and
-   * {@link #enableTable(org.apache.hadoop.hbase.TableName)}
-   *
-   * @param pattern The pattern to match table names against
-   * @throws IOException
-   * @return Table descriptors for tables that couldn't be enabled.
-   *         The return HTDs are read-only.
-   * @deprecated since 2.0 version and will be removed in 3.0 version
-   *             This is just a trivial helper method without any magic.
-   *             Consider using {@link #listTableDescriptors(java.util.regex.Pattern)}
-   *             and {@link #enableTable(org.apache.hadoop.hbase.TableName)}
-   */
-  @Deprecated
-  HTableDescriptor[] enableTables(Pattern pattern) throws IOException;
-
-  /**
    * Disable the table but does not block and wait for it to be completely disabled.
    * 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
@@ -548,43 +357,6 @@ public interface Admin extends Abortable, Closeable {
   }
 
   /**
-   * Disable tables matching the passed in pattern and wait on completion. Warning: Use this method
-   * carefully, there is no prompting and the effect is immediate. Consider using {@link
-   * #listTableDescriptors(Pattern)} and {@link #disableTable(org.apache.hadoop.hbase.TableName)}
-   *
-   * @param regex The regular expression to match table names against
-   * @return Table descriptors for tables that couldn't be disabled
-   *         The return htds are read-only
-   * @throws IOException
-   * @see #disableTables(java.util.regex.Pattern)
-   * @see #disableTable(org.apache.hadoop.hbase.TableName)
-   * @deprecated since 2.0 version and will be removed in 3.0 version
-   *             This is just a trivial helper method without any magic.
-   *             Consider using {@link #listTableDescriptors(Pattern)}
-   *             and {@link #disableTable(org.apache.hadoop.hbase.TableName)}
-   */
-  @Deprecated
-  HTableDescriptor[] disableTables(String regex) throws IOException;
-
-  /**
-   * Disable tables matching the passed in pattern and wait on completion. Warning: Use this method
-   * carefully, there is no prompting and the effect is immediate. Consider using {@link
-   * #listTableDescriptors(java.util.regex.Pattern)} and
-   * {@link #disableTable(org.apache.hadoop.hbase.TableName)}
-   *
-   * @param pattern The pattern to match table names against
-   * @return Table descriptors for tables that couldn't be disabled
-   *         The return htds are read-only
-   * @throws IOException
-   * @deprecated since 2.0 version and will be removed in 3.0 version
-   *             This is just a trivial helper method without any magic.
-   *             Consider using {@link #listTableDescriptors(java.util.regex.Pattern)}
-   *             and {@link #disableTable(org.apache.hadoop.hbase.TableName)}
-   */
-  @Deprecated
-  HTableDescriptor[] disableTables(Pattern pattern) throws IOException;
-
-  /**
    * @param tableName name of table to check
    * @return <code>true</code> if table is on-line
    * @throws IOException if a remote or network exception occurs
@@ -606,69 +378,9 @@ public interface Admin extends Abortable, Closeable {
   boolean isTableAvailable(TableName tableName) throws IOException;
 
   /**
-   * Use this api to check if the table has been created with the specified number of splitkeys
-   * which was used while creating the given table. Note : If this api is used after a table's
-   * region gets splitted, the api may return <code>false</code>.
-   *
-   * @param tableName name of table to check
-   * @param splitKeys keys to check if the table has been created with all split keys
-   * @throws IOException if a remote or network excpetion occurs
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #isTableAvailable(TableName)}
-   */
-  @Deprecated
-  boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException;
-
-  /**
-   * Get the status of an <code>alter</code> (a.k.a <code>modify</code>) command - indicates how
-   * many regions have received the updated schema Asynchronous operation.
-   *
-   * @param tableName TableName instance
-   * @return Pair indicating the number of regions updated Pair.getFirst() is the regions that are
-   * yet to be updated Pair.getSecond() is the total number of regions of the table
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. No longer needed now you get a Future
-   * on an operation.
-   */
-  @Deprecated
-  Pair<Integer, Integer> getAlterStatus(TableName tableName) throws IOException;
-
-  /**
-   * Get the status of <code>alter</code> (a.k.a <code>modify</code>) command - indicates how many
-   * regions have received the updated schema Asynchronous operation.
-   *
-   * @param tableName name of the table to get the status of
-   * @return Pair indicating the number of regions updated Pair.getFirst() is the regions that are
-   * yet to be updated Pair.getSecond() is the total number of regions of the table
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. No longer needed now you get a Future
-   * on an operation.
-   */
-  @Deprecated
-  Pair<Integer, Integer> getAlterStatus(byte[] tableName) throws IOException;
-
-  /**
-   * Add a column family to an existing table. Synchronous operation.
-   * Use {@link #addColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it
-   * returns a {@link Future} from which you can learn whether success or failure.
-   *
-   * @param tableName name of the table to add column family to
-   * @param columnFamily column family descriptor of column family to be added
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #addColumnFamily(TableName, ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  default void addColumn(TableName tableName, ColumnFamilyDescriptor columnFamily)
-    throws IOException {
-    addColumnFamily(tableName, columnFamily);
-  }
-
-  /**
-   * Add a column family to an existing table. Synchronous operation.
-   * Use {@link #addColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it
-   * returns a {@link Future} from which you can learn whether success or failure.
-   *
+   * Add a column family to an existing table. Synchronous operation. Use
+   * {@link #addColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it returns a
+   * {@link Future} from which you can learn whether success or failure.
    * @param tableName name of the table to add column family to
    * @param columnFamily column family descriptor of column family to be added
    * @throws IOException if a remote or network exception occurs
@@ -695,24 +407,9 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
-   * Delete a column family from a table. Synchronous operation.
-   *  Use {@link #deleteColumnFamily(TableName, byte[])} instead because it
-   * returns a {@link Future} from which you can learn whether success or failure.
-   *
-   * @param tableName name of table
-   * @param columnFamily name of column family to be deleted
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #deleteColumnFamily(TableName, byte[])}}.
-   */
-  @Deprecated
-  void deleteColumn(TableName tableName, byte[] columnFamily) throws IOException;
-
-  /**
-   * Delete a column family from a table. Synchronous operation.
-   * Use {@link #deleteColumnFamily(TableName, byte[])} instead because it
-   * returns a {@link Future} from which you can learn whether success or failure.
+   * Delete a column family from a table. Synchronous operation. Use
+   * {@link #deleteColumnFamily(TableName, byte[])} instead because it returns a {@link Future} from
+   * which you can learn whether success or failure.
    * @param tableName name of table
    * @param columnFamily name of column family to be deleted
    * @throws IOException if a remote or network exception occurs
@@ -745,23 +442,6 @@ public interface Admin extends Abortable, Closeable {
    * @param tableName name of table
    * @param columnFamily new column family descriptor to use
    * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #modifyColumnFamily(TableName, ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  default void modifyColumn(TableName tableName, ColumnFamilyDescriptor columnFamily)
-      throws IOException {
-    modifyColumnFamily(tableName, columnFamily);
-  }
-
-  /**
-   * Modify an existing column family on a table. Synchronous operation.
-   * Use {@link #modifyColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it
-   * returns a {@link Future} from which you can learn whether success or failure.
-   * @param tableName name of table
-   * @param columnFamily new column family descriptor to use
-   * @throws IOException if a remote or network exception occurs
    */
   default void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
       throws IOException {
@@ -786,67 +466,6 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
-   * Uses {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins.
-   *
-   * @param regionname region name to close
-   * @param serverName Deprecated. Not used.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   *             Use {@link #unassign(byte[], boolean)}.
-   */
-  @Deprecated
-  void closeRegion(String regionname, String serverName) throws IOException;
-
-  /**
-   * Uses {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins.
-   *
-   * @param regionname region name to close
-   * @param serverName Deprecated. Not used.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   *             Use {@link #unassign(byte[], boolean)}.
-   */
-  @Deprecated
-  void closeRegion(byte[] regionname, String serverName) throws IOException;
-
-  /**
-   * Uses {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins.
-   *
-   * @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 serverName Deprecated. Not used.
-   * @return Deprecated. Returns <code>true</code> always.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   *             Use {@link #unassign(byte[], boolean)}.
-   */
-  @Deprecated
-  boolean closeRegionWithEncodedRegionName(String encodedRegionName, String serverName)
-      throws IOException;
-
-  /**
-   * Used {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins.
-   *
-   * @param sn Deprecated. Not used.
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18231">HBASE-18231</a>).
-   *             Use {@link #unassign(byte[], boolean)}.
-   */
-  @Deprecated
-  void closeRegion(final ServerName sn, final HRegionInfo hri) throws IOException;
-
-  /**
-   * Get all the online regions on a region server.
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-17980">HBASE-17980</a>).
-   *             Use {@link #getRegions(ServerName sn)}.
-   */
-  @Deprecated
-  List<HRegionInfo> getOnlineRegions(ServerName sn) throws IOException;
-
-  /**
    * Get all the online regions on a region server.
    *
    * @return List of {@link RegionInfo}
@@ -1020,28 +639,6 @@ public interface Admin extends Abortable, Closeable {
     throws IOException, InterruptedException;
 
   /**
-   * Compact all regions on the region server. Asynchronous operation in that this method requests
-   * that a Compaction run and then it returns. It does not wait on the completion of Compaction (it
-   * can take a while).
-   * @param sn the region server name
-   * @param major if it's major compaction
-   * @throws IOException if a remote or network exception occurs
-   * @throws InterruptedException
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
-   *             {@link #compactRegionServer(ServerName)} or
-   *             {@link #majorCompactRegionServer(ServerName)}.
-   */
-  @Deprecated
-  default void compactRegionServer(ServerName sn, boolean major) throws IOException,
-      InterruptedException {
-    if (major) {
-      majorCompactRegionServer(sn);
-    } else {
-      compactRegionServer(sn);
-    }
-  }
-
-  /**
    * Turn the compaction on or off. Disabling compactions will also interrupt any currently ongoing
    * compactions. This state is ephemeral. The setting will be lost on restart. Compaction
    * can also be enabled/disabled by modifying configuration hbase.regionserver.compaction.enabled
@@ -1119,40 +716,13 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Turn the load balancer on or off.
-   *
-   * @param synchronous If <code>true</code>, it waits until current balance() call, if
-   * outstanding, to return.
-   * @return Previous balancer value
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0.
-   * Use {@link #balancerSwitch(boolean, boolean)} instead.
-   */
-  @Deprecated
-  default boolean setBalancerRunning(boolean on, boolean synchronous) throws IOException {
-    return balancerSwitch(on, synchronous);
-  }
-
-  /**
-   * Turn the load balancer on or off.
    * @param onOrOff Set to <code>true</code> to enable, <code>false</code> to disable.
-   * @param synchronous If <code>true</code>, it waits until current balance() call, if
-   * outstanding, to return.
+   * @param synchronous If <code>true</code>, it waits until current balance() call, if outstanding,
+   *          to return.
    * @return Previous balancer value
    */
-  boolean balancerSwitch(boolean onOrOff, boolean synchronous)
-  throws IOException;
+  boolean balancerSwitch(boolean onOrOff, boolean synchronous) throws IOException;
 
-  /**
-   * Invoke the balancer.  Will run the balancer and if regions to move, it will go ahead and do the
-   * reassignments.  Can NOT run for various reasons.  Check logs.
-   *
-   * @return <code>true</code> if balancer ran, <code>false</code> otherwise.
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0.
-   * Use {@link #balance()} instead.
-   */
-  @Deprecated
-  default boolean balancer() throws IOException {
-    return balance();
-  }
 
   /**
    * Invoke the balancer.  Will run the balancer and if regions to move, it will go ahead and do the
@@ -1169,21 +739,6 @@ public interface Admin extends Abortable, Closeable {
    * logs.
    * @param force whether we should force balance even if there is region in transition
    * @return <code>true</code> if balancer ran, <code>false</code> otherwise.
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0.
-   * Use {@link #balance(boolean)} instead.
-   */
-  @Deprecated
-  default boolean balancer(boolean force) throws IOException {
-    return balance(force);
-  }
-
-  /**
-   * Invoke the balancer.  Will run the balancer and if regions to move, it will
-   * go ahead and do the reassignments. If there is region in transition, force parameter of true
-   * would still run balancer. Can *not* run for other reasons.  Check
-   * logs.
-   * @param force whether we should force balance even if there is region in transition
-   * @return <code>true</code> if balancer ran, <code>false</code> otherwise.
    */
   boolean balance(boolean force) throws IOException;
 
@@ -1224,33 +779,8 @@ public interface Admin extends Abortable, Closeable {
    * Turn region normalizer on or off.
    *
    * @return Previous normalizer value
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #normalizerSwitch(boolean)}}
-   * instead.
-   */
-  @Deprecated
-  default boolean setNormalizerRunning(boolean on) throws IOException {
-    return normalizerSwitch(on);
-  }
-
-  /**
-   * Turn region normalizer on or off.
-   *
-   * @return Previous normalizer value
-   */
-  boolean normalizerSwitch (boolean on) throws IOException;
-
-  /**
-   * Enable/Disable the catalog janitor.
-   *
-   * @param enable if <code>true</code> enables the catalog janitor
-   * @return the previous state
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #catalogJanitorSwitch(boolean)}}
-   * instead.
    */
-  @Deprecated
-  default boolean enableCatalogJanitor(boolean enable) throws IOException {
-    return catalogJanitorSwitch(enable);
-  }
+  boolean normalizerSwitch(boolean on) throws IOException;
 
   /**
    * Enable/Disable the catalog janitor/
@@ -1264,18 +794,6 @@ public interface Admin extends Abortable, Closeable {
    * Ask for a scan of the catalog table.
    *
    * @return the number of entries cleaned
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #runCatalogJanitor()}}
-   * instead.
-   */
-  @Deprecated
-  default int runCatalogScan() throws IOException {
-    return runCatalogJanitor();
-  }
-
-  /**
-   * Ask for a scan of the catalog table.
-   *
-   * @return the number of entries cleaned
    */
   int runCatalogJanitor() throws IOException;
 
@@ -1288,20 +806,6 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Enable/Disable the cleaner chore.
    *
-   * @param on if <code>true</code> enables the cleaner chore
-   * @return the previous state
-   * @throws IOException
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #cleanerChoreSwitch(boolean)}}
-   * instead.
-   */
-  @Deprecated
-  default boolean setCleanerChoreRunning(boolean on) throws IOException {
-    return cleanerChoreSwitch(on);
-  }
-
-  /**
-   * Enable/Disable the cleaner chore.
-   *
    * @param onOrOff if <code>true</code> enables the cleaner chore
    * @return the previous state
    * @throws IOException
@@ -1323,21 +827,6 @@ public interface Admin extends Abortable, Closeable {
    */
   boolean isCleanerChoreEnabled() throws IOException;
 
-  /**
-   * Merge two regions. Asynchronous operation.
-   *
-   * @param nameOfRegionA encoded or full name of region a
-   * @param nameOfRegionB encoded or full name of region b
-   * @param forcible <code>true</code> if do a compulsory merge, otherwise we will only merge two
-   * adjacent regions
-   * @throws IOException
-   * @deprecated Since 2.0. Will be removed in 3.0. Use
-   *     {@link #mergeRegionsAsync(byte[], byte[], boolean)} instead.
-   */
-  @Deprecated
-  void mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB,
-      boolean forcible) throws IOException;
-
 
   /**
    * Merge two regions. Asynchronous operation.
@@ -1374,17 +863,6 @@ public interface Admin extends Abortable, Closeable {
   void split(TableName tableName) throws IOException;
 
   /**
-   * Split an individual region. Asynchronous operation.
-   *
-   * @param regionName region to split
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   *             Use {@link #splitRegionAsync(byte[], byte[])}.
-   */
-  @Deprecated
-  void splitRegion(byte[] regionName) throws IOException;
-
-  /**
    * Split a table. Asynchronous operation.
    *
    * @param tableName table to split
@@ -1395,19 +873,6 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Split an individual region. Asynchronous operation.
-   *
-   * @param regionName region to split
-   * @param splitPoint the explicit position to split on
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   *             Use {@link #splitRegionAsync(byte[], byte[])}.
-   */
-  @Deprecated
-  void splitRegion(byte[] regionName, byte[] splitPoint)
-    throws IOException;
-
-  /**
-   * Split an individual region. Asynchronous operation.
    * @param regionName region to split
    * @param splitPoint the explicit position to split on
    * @throws IOException if a remote or network exception occurs
@@ -1416,23 +881,6 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Modify an existing table, more IRB friendly version.
-   * @param tableName name of table.
-   * @param td modified description of the table
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version. use
-   *             {@link #modifyTable(TableDescriptor)}
-   */
-  @Deprecated
-  default void modifyTable(TableName tableName, TableDescriptor td) throws IOException {
-    if (!tableName.equals(td.getTableName())) {
-      throw new IllegalArgumentException("the specified table name '" + tableName +
-        "' doesn't match with the HTD one: " + td.getTableName());
-    }
-    modifyTable(td);
-  }
-
-  /**
-   * Modify an existing table, more IRB friendly version.
    * @param td modified description of the table
    * @throws IOException if a remote or network exception occurs
    */
@@ -1441,39 +889,11 @@ public interface Admin extends Abortable, Closeable {
   }
 
   /**
-   * Modify an existing table, more IRB friendly version. Asynchronous operation.  This means that
-   * it may be a while before your schema change is updated across all of the table.
-   * 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.
-   *
-   * @param tableName name of table.
-   * @param td modified description of the table
-   * @throws IOException if a remote or network exception occurs
-   * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
-   *     operation to complete
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #modifyTableAsync(TableDescriptor)}
-   */
-  @Deprecated
-  default Future<Void> modifyTableAsync(TableName tableName, TableDescriptor td)
-      throws IOException {
-    if (!tableName.equals(td.getTableName())) {
-      throw new IllegalArgumentException("the specified table name '" + tableName +
-        "' doesn't match with the HTD one: " + td.getTableName());
-    }
-    return modifyTableAsync(td);
-  }
-
-  /**
-   * Modify an existing table, more IRB (ruby) friendly version. Asynchronous operation. This means that
-   * it may be a while before your schema change is updated across all of the table.
-   * 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.
-   *
+   * Modify an existing table, more IRB (ruby) friendly version. Asynchronous operation. This means
+   * that it may be a while before your schema change is updated across all of the table. 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.
    * @param td description of the table
    * @throws IOException if a remote or network exception occurs
    * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
@@ -1517,27 +937,6 @@ public interface Admin extends Abortable, Closeable {
   void stopRegionServer(String hostnamePort) throws IOException;
 
   /**
-   * Get whole cluster status, containing status about:
-   * <pre>
-   * hbase version
-   * cluster id
-   * primary/backup master(s)
-   * master's coprocessors
-   * live/dead regionservers
-   * balancer
-   * regions in transition
-   * </pre>
-   * @return cluster status
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #getClusterMetrics()}
-   */
-  @Deprecated
-  default ClusterStatus getClusterStatus() throws IOException {
-    return new ClusterStatus(getClusterMetrics());
-  }
-
-  /**
    * Get whole cluster metrics, containing status about:
    * <pre>
    * hbase version
@@ -1676,24 +1075,9 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * List available namespace descriptors.
-   *
    * @return List of descriptors
    */
-  NamespaceDescriptor[] listNamespaceDescriptors()
-  throws IOException;
-
-  /**
-   * Get list of table descriptors by namespace.
-   *
-   * @param name namespace name
-   * @return HTD[] the read-only tableDescriptors
-   * @throws IOException
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #listTableDescriptorsByNamespace(byte[])}
-   */
-  @Deprecated
-  HTableDescriptor[] listTableDescriptorsByNamespace(String name)
-      throws IOException;
+  NamespaceDescriptor[] listNamespaceDescriptors() throws IOException;
 
   /**
    * Get list of table descriptors by namespace.
@@ -1713,40 +1097,12 @@ public interface Admin extends Abortable, Closeable {
    * Get the regions of a given table.
    *
    * @param tableName the name of the table
-   * @return List of {@link HRegionInfo}.
-   * @throws IOException
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-17980">HBASE-17980</a>).
-   *             Use {@link #getRegions(TableName)}.
-   */
-  @Deprecated
-  List<HRegionInfo> getTableRegions(TableName tableName)
-    throws IOException;
-
-  /**
-   * Get the regions of a given table.
-   *
-   * @param tableName the name of the table
    * @return List of {@link RegionInfo}.
-   * @throws IOException
    */
   List<RegionInfo> getRegions(TableName tableName) throws IOException;
 
   @Override
-  void close() throws IOException;
-
-  /**
-   * Get tableDescriptors.
-   *
-   * @param tableNames List of table names
-   * @return HTD[] the read-only tableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #listTableDescriptors(List)}
-   */
-  @Deprecated
-  HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames)
-    throws IOException;
+  void close();
 
   /**
    * Get tableDescriptors.
@@ -1759,19 +1115,6 @@ public interface Admin extends Abortable, Closeable {
     throws IOException;
 
   /**
-   * Get tableDescriptors.
-   *
-   * @param names List of table names
-   * @return HTD[] the read-only tableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #listTableDescriptors(List)}
-   */
-  @Deprecated
-  HTableDescriptor[] getTableDescriptors(List<String> names)
-    throws IOException;
-
-  /**
    * Abort a procedure.
    * <p/>
    * Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.
@@ -1803,9 +1146,8 @@ public interface Admin extends Abortable, Closeable {
    * @deprecated Since 2.1.1 -- to be removed.
    */
   @Deprecated
-  Future<Boolean> abortProcedureAsync(
-    long procId,
-    boolean mayInterruptIfRunning) throws IOException;
+  Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning)
+      throws IOException;
 
   /**
    * Get procedures.
@@ -1838,18 +1180,6 @@ public interface Admin extends Abortable, Closeable {
    * Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
    * @return an array of master coprocessors
    * @see org.apache.hadoop.hbase.ClusterMetrics#getMasterCoprocessorNames()
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #getMasterCoprocessorNames()}
-   */
-  @Deprecated
-  default String[] getMasterCoprocessors() throws IOException {
-    return getMasterCoprocessorNames().stream().toArray(size -> new String[size]);
-  }
-
-  /**
-   * Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
-   * @return an array of master coprocessors
-   * @see org.apache.hadoop.hbase.ClusterMetrics#getMasterCoprocessorNames()
    */
   default List<String> getMasterCoprocessorNames() throws IOException {
     return getClusterMetrics(EnumSet.of(Option.MASTER_COPROCESSORS))
@@ -1931,26 +1261,6 @@ public interface Admin extends Abortable, Closeable {
   }
 
   /**
-   * Create a timestamp consistent snapshot for the given table. Snapshots are considered unique
-   * based on <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even
-   * different type or with different parameters) will fail with a {@link SnapshotCreationException}
-   * indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in
-   * HBase.
-   * @param snapshotName name of the snapshot to be created
-   * @param tableName name of the table for which snapshot is created
-   * @throws IOException if a remote or network exception occurs
-   * @throws SnapshotCreationException if snapshot creation failed
-   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
-   * @deprecated since 2.3.0, will be removed in 3.0.0. Use {@link #snapshot(String, TableName)}
-   *             instead.
-   */
-  @Deprecated
-  default void snapshot(byte[] snapshotName, TableName tableName)
-      throws IOException, SnapshotCreationException, IllegalArgumentException {
-    snapshot(Bytes.toString(snapshotName), tableName);
-  }
-
-  /**
    * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the
    * snapshot</b>. Attempts to take a snapshot with the same name (even a different type or with
    * different parameters) will fail with a {@link SnapshotCreationException} indicating the
@@ -1978,9 +1288,8 @@ public interface Admin extends Abortable, Closeable {
    * parameters) will fail with a {@link SnapshotCreationException} indicating the duplicate naming.
    * Snapshot names follow the same naming constraints as tables in HBase. See
    * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. You should
-   * probably use {@link #snapshot(String, org.apache.hadoop.hbase.TableName)} or
-   * {@link #snapshot(byte[], org.apache.hadoop.hbase.TableName)} unless you are sure about the type
-   * of snapshot that you want to take.
+   * probably use {@link #snapshot(String, org.apache.hadoop.hbase.TableName)} unless you are sure
+   * about the type of snapshot that you want to take.
    * @param snapshot snapshot to take
    * @throws IOException or we lose contact with the master.
    * @throws SnapshotCreationException if snapshot failed to be taken
@@ -1997,23 +1306,6 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if the snapshot did not succeed or we lose contact with the master.
    * @throws SnapshotCreationException if snapshot creation failed
    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
-   * {@link #snapshotAsync(SnapshotDescription)} instead.
-   */
-  @Deprecated
-  @SuppressWarnings("FutureReturnValueIgnored")
-  default void takeSnapshotAsync(SnapshotDescription snapshot)
-      throws IOException, SnapshotCreationException {
-    snapshotAsync(snapshot);
-  }
-
-  /**
-   * Take a snapshot without waiting for the server to complete that snapshot (asynchronous) Only a
-   * single snapshot should be taken at a time, or results may be undefined.
-   * @param snapshot snapshot to take
-   * @throws IOException if the snapshot did not succeed or we lose contact with the master.
-   * @throws SnapshotCreationException if snapshot creation failed
-   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
    */
   Future<Void> snapshotAsync(SnapshotDescription snapshot)
       throws IOException, SnapshotCreationException;
@@ -2045,70 +1337,12 @@ public interface Admin extends Abortable, Closeable {
    * completes without problem the failsafe snapshot is deleted.
    * @param snapshotName name of the snapshot to restore
    * @throws IOException if a remote or network exception occurs
-   * @throws org.apache.hadoop.hbase.snapshot.RestoreSnapshotException if snapshot failed to be
-   *           restored
-   * @throws IllegalArgumentException if the restore request is formatted incorrectly
-   * @deprecated since 2.3.0, will be removed in 3.0.0. Use {@link #restoreSnapshot(String)}
-   *             instead.
-   */
-  @Deprecated
-  default void restoreSnapshot(byte[] snapshotName) throws IOException, RestoreSnapshotException {
-    restoreSnapshot(Bytes.toString(snapshotName));
-  }
-
-  /**
-   * Restore the specified snapshot on the original table. (The table must be disabled) If the
-   * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to
-   * <code>true</code>, a snapshot of the current table is taken before executing the restore
-   * operation. In case of restore failure, the failsafe snapshot will be restored. If the restore
-   * completes without problem the failsafe snapshot is deleted.
-   * @param snapshotName name of the snapshot to restore
-   * @throws IOException if a remote or network exception occurs
    * @throws RestoreSnapshotException if snapshot failed to be restored
    * @throws IllegalArgumentException if the restore request is formatted incorrectly
    */
   void restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException;
 
   /**
-   * Restore the specified snapshot on the original table. (The table must be disabled) If the
-   * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to
-   * <code>true</code>, a snapshot of the current table is taken before executing the restore
-   * operation. In case of restore failure, the failsafe snapshot will be restored. If the restore
-   * completes without problem the failsafe snapshot is deleted.
-   * @param snapshotName name of the snapshot to restore
-   * @throws IOException if a remote or network exception occurs
-   * @throws RestoreSnapshotException if snapshot failed to be restored
-   * @return the result of the async restore snapshot. You can use Future.get(long, TimeUnit) to
-   *         wait on the operation to complete.
-   * @deprecated since 2.3.0, will be removed in 3.0.0. The implementation does not take care of the
-   *             failsafe property, so do not use it any more.
-   */
-  @Deprecated
-  Future<Void> restoreSnapshotAsync(String snapshotName)
-      throws IOException, RestoreSnapshotException;
-
-  /**
-   * Restore the specified snapshot on the original table. (The table must be disabled) If
-   * 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken
-   * before executing the restore operation. In case of restore failure, the failsafe snapshot will
-   * be restored. If the restore completes without problem the failsafe snapshot is deleted. The
-   * failsafe snapshot name is configurable by using the property
-   * "hbase.snapshot.restore.failsafe.name".
-   * @param snapshotName name of the snapshot to restore
-   * @param takeFailSafeSnapshot <code>true</code> if the failsafe snapshot should be taken
-   * @throws IOException if a remote or network exception occurs
-   * @throws RestoreSnapshotException if snapshot failed to be restored
-   * @throws IllegalArgumentException if the restore request is formatted incorrectly
-   * @deprecated since 2.3.0, will be removed in 3.0.0. Use
-   *             {@link #restoreSnapshot(String, boolean)} instead.
-   */
-  @Deprecated
-  default void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot)
-      throws IOException, RestoreSnapshotException {
-    restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
-  }
-
-  /**
    * Restore the specified snapshot on the original table. (The table must be disabled) If
    * 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken
    * before executing the restore operation. In case of restore failure, the failsafe snapshot will
@@ -2140,25 +1374,8 @@ public interface Admin extends Abortable, Closeable {
    * @throws RestoreSnapshotException if snapshot failed to be restored
    * @throws IllegalArgumentException if the restore request is formatted incorrectly
    */
-  void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot,
-      boolean restoreAcl) throws IOException, RestoreSnapshotException;
-
-  /**
-   * Create a new table by cloning the snapshot content.
-   * @param snapshotName name of the snapshot to be cloned
-   * @param tableName name of the table where the snapshot will be restored
-   * @throws IOException if a remote or network exception occurs
-   * @throws TableExistsException if table to be created already exists
-   * @throws RestoreSnapshotException if snapshot failed to be cloned
-   * @throws IllegalArgumentException if the specified table has not a valid name
-   * @deprecated since 2.3.0, will be removed in 3.0.0. Use
-   *             {@link #cloneSnapshot(String, TableName)} instead.
-   */
-  @Deprecated
-  default void cloneSnapshot(byte[] snapshotName, TableName tableName)
-      throws IOException, TableExistsException, RestoreSnapshotException {
-    cloneSnapshot(Bytes.toString(snapshotName), tableName);
-  }
+  void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl)
+      throws IOException, RestoreSnapshotException;
 
   /**
    * Create a new table by cloning the snapshot content.
@@ -2244,24 +1461,6 @@ public interface Admin extends Abortable, Closeable {
    * @param props Property/Value pairs of properties passing to the procedure
    * @return data returned after procedure execution. null if no return data.
    * @throws IOException
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
-   * {@link #execProcedureWithReturn(String, String, Map)} } instead.
-   */
-  @Deprecated
-  default byte[] execProcedureWithRet(String signature, String instance, Map<String, String> props)
-      throws IOException {
-    return execProcedureWithReturn(signature, instance, props);
-  }
-
-  /**
-   * Execute a distributed procedure on a cluster.
-   * @param signature A distributed procedure is uniquely identified by its signature (default the
-   *          root ZK node name of the procedure).
-   * @param instance The instance name of the procedure. For some procedures, this parameter is
-   *          optional.
-   * @param props Property/Value pairs of properties passing to the procedure
-   * @return data returned after procedure execution. null if no return data.
-   * @throws IOException
    */
   byte[] execProcedureWithReturn(String signature, String instance, Map<String, String> props)
       throws IOException;
@@ -2289,18 +1488,6 @@ public interface Admin extends Abortable, Closeable {
   List<SnapshotDescription> listSnapshots() throws IOException;
 
   /**
-   * List all the completed snapshots matching the given regular expression.
-   *
-   * @param regex The regular expression to match against
-   * @return list of SnapshotDescription
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #listSnapshots(Pattern)} instead.
-   */
-  @Deprecated
-  List<SnapshotDescription> listSnapshots(String regex) throws IOException;
-
-  /**
    * List all the completed snapshots matching the given pattern.
    *
    * @param pattern The compiled regular expression to match against
@@ -2312,20 +1499,6 @@ public interface Admin extends Abortable, Closeable {
   /**
    * List all the completed snapshots matching the given table name regular expression and snapshot
    * name regular expression.
-   * @param tableNameRegex The table name regular expression to match against
-   * @param snapshotNameRegex The snapshot name regular expression to match against
-   * @return list of completed SnapshotDescription
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #listTableSnapshots(Pattern, Pattern)} instead.
-   */
-  @Deprecated
-  List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
-      String snapshotNameRegex) throws IOException;
-
-  /**
-   * List all the completed snapshots matching the given table name regular expression and snapshot
-   * name regular expression.
    * @param tableNamePattern The compiled table name regular expression to match against
    * @param snapshotNamePattern The compiled snapshot name regular expression to match against
    * @return list of completed SnapshotDescription
@@ -2353,17 +1526,6 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Delete existing snapshots whose names match the pattern passed.
    *
-   * @param regex The regular expression to match against
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #deleteSnapshots(Pattern)} instead.
-   */
-  @Deprecated
-  void deleteSnapshots(String regex) throws IOException;
-
-  /**
-   * Delete existing snapshots whose names match the pattern passed.
-   *
    * @param pattern pattern for names of the snapshot to match
    * @throws IOException if a remote or network exception occurs
    */
@@ -2372,18 +1534,6 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Delete all existing snapshots matching the given table name regular expression and snapshot
    * name regular expression.
-   * @param tableNameRegex The table name regular expression to match against
-   * @param snapshotNameRegex The snapshot name regular expression to match against
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #deleteTableSnapshots(Pattern, Pattern)} instead.
-   */
-  @Deprecated
-  void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException;
-
-  /**
-   * Delete all existing snapshots matching the given table name regular expression and snapshot
-   * name regular expression.
    * @param tableNamePattern The compiled table name regular expression to match against
    * @param snapshotNamePattern The compiled snapshot name regular expression to match against
    * @throws IOException if a remote or network exception occurs
@@ -2400,16 +1550,6 @@ public interface Admin extends Abortable, Closeable {
   void setQuota(QuotaSettings quota) throws IOException;
 
   /**
-   * Return a QuotaRetriever to list the quotas based on the filter.
-   * @param filter the quota settings filter
-   * @return the quota retriever
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #getQuota(QuotaFilter)}.
-   */
-  @Deprecated
-  QuotaRetriever getQuotaRetriever(QuotaFilter filter) throws IOException;
-
-  /**
    * List the quotas based on the filter.
    * @param filter the quota settings filter
    * @return the QuotaSetting list
@@ -2494,35 +1634,6 @@ public interface Admin extends Abortable, Closeable {
   List<SecurityCapability> getSecurityCapabilities() throws IOException;
 
   /**
-   * Turn the Split or Merge switches on or off.
-   * @param enabled enabled or not
-   * @param synchronous If <code>true</code>, it waits until current split() call, if outstanding,
-   *          to return.
-   * @param switchTypes switchType list {@link MasterSwitchType}
-   * @return Previous switch value array
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #splitSwitch(boolean, boolean)}
-   *             or {@link #mergeSwitch(boolean, boolean)} instead.
-   */
-  @Deprecated
-  default boolean[] setSplitOrMergeEnabled(boolean enabled, boolean synchronous,
-      MasterSwitchType... switchTypes) throws IOException {
-    boolean[] preValues = new boolean[switchTypes.length];
-    for (int i = 0; i < switchTypes.length; i++) {
-      switch (switchTypes[i]) {
-        case SPLIT:
-          preValues[i] = splitSwitch(enabled, synchronous);
-          break;
-        case MERGE:
-          preValues[i] = mergeSwitch(enabled, synchronous);
-          break;
-        default:
-          throw new UnsupportedOperationException("Unsupported switch type:" + switchTypes[i]);
-      }
-    }
-    return preValues;
-  }
-
-  /**
    * Turn the split switch on or off.
    * @param enabled enabled or not
    * @param synchronous If <code>true</code>, it waits until current split() call, if outstanding,
@@ -2541,26 +1652,6 @@ public interface Admin extends Abortable, Closeable {
   boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException;
 
   /**
-   * Query the current state of the switch.
-   *
-   * @return <code>true</code> if the switch is enabled, <code>false</code> otherwise.
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
-   * {@link #isSplitEnabled()} or {@link #isMergeEnabled()} instead.
-   */
-  @Deprecated
-  default boolean isSplitOrMergeEnabled(MasterSwitchType switchType) throws IOException {
-    switch (switchType) {
-      case SPLIT:
-        return isSplitEnabled();
-      case MERGE:
-        return isMergeEnabled();
-      default:
-        break;
-    }
-    throw new UnsupportedOperationException("Unsupported switch type:" + switchType);
-  }
-
-  /**
    * Query the current state of the split switch.
    * @return <code>true</code> if the switch is enabled, <code>false</code> otherwise.
    */
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 1bfb7b3..3a0588b 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
@@ -45,7 +45,6 @@ import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
 import org.apache.hadoop.hbase.ClusterMetrics;
@@ -54,9 +53,7 @@ import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -136,7 +133,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Coprocesso
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
@@ -170,8 +166,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedu
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
@@ -462,49 +456,13 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] listTables() throws IOException {
-    return listTables((Pattern)null, false);
-  }
-
-  @Override
-  public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
-    return listTables(pattern, false);
-  }
-
-  @Override
-  public HTableDescriptor[] listTables(String regex) throws IOException {
-    return listTables(Pattern.compile(regex), false);
-  }
-
-  @Override
-  public HTableDescriptor[] listTables(final Pattern pattern, final boolean includeSysTables)
-      throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected HTableDescriptor[] rpcCall() throws Exception {
-        GetTableDescriptorsRequest req =
-            RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
-        return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(),
-                req)).stream().map(ImmutableHTableDescriptor::new).toArray(HTableDescriptor[]::new);
-      }
-    });
-  }
-
-  @Override
-  public HTableDescriptor[] listTables(String regex, boolean includeSysTables)
-      throws IOException {
-    return listTables(Pattern.compile(regex), includeSysTables);
-  }
-
-  @Override
   public TableName[] listTableNames() throws IOException {
     return listTableNames((Pattern)null, false);
   }
 
   @Override
-  public TableName[] listTableNames(String regex) throws IOException {
-    return listTableNames(Pattern.compile(regex), false);
+  public TableName[] listTableNames(Pattern pattern) throws IOException {
+    return listTableNames(pattern, false);
   }
 
   @Override
@@ -522,18 +480,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  @Override
-  public TableName[] listTableNames(final String regex, final boolean includeSysTables)
-      throws IOException {
-    return listTableNames(Pattern.compile(regex), includeSysTables);
-  }
-
-  @Override
-  public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
-    return getHTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
-       operationTimeout, rpcTimeout);
-  }
-
   static TableDescriptor getTableDescriptor(final TableName tableName, Connection connection,
       RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
       int operationTimeout, int rpcTimeout) throws IOException {
@@ -557,38 +503,6 @@ public class HBaseAdmin implements Admin {
     throw new TableNotFoundException(tableName.getNameAsString());
   }
 
-  /**
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #getTableDescriptor(TableName,
-   *             Connection, RpcRetryingCallerFactory,RpcControllerFactory,int,int)}
-   */
-  @Deprecated
-  static HTableDescriptor getHTableDescriptor(final TableName tableName, Connection connection,
-      RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
-      int operationTimeout, int rpcTimeout) throws IOException {
-    if (tableName == null) {
-      return null;
-    }
-    HTableDescriptor htd =
-        executeCallable(new MasterCallable<HTableDescriptor>(connection, rpcControllerFactory) {
-          @Override
-          protected HTableDescriptor rpcCall() throws Exception {
-            GetTableDescriptorsRequest req =
-                RequestConverter.buildGetTableDescriptorsRequest(tableName);
-            GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
-            if (!htds.getTableSchemaList().isEmpty()) {
-              return new ImmutableHTableDescriptor(
-                  ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0)));
-            }
-            return null;
-          }
-        }, rpcCallerFactory, operationTimeout, rpcTimeout);
-    if (htd != null) {
-      return new ImmutableHTableDescriptor(htd);
-    }
-    throw new TableNotFoundException(tableName.getNameAsString());
-  }
-
   private long getPauseTime(int tries) {
     int triesCount = tries;
     if (triesCount >= HConstants.RETRY_BACKOFF.length) {
@@ -673,7 +587,7 @@ public class HBaseAdmin implements Admin {
     }
 
     @Override
-    protected TableDescriptor getTableDescriptor() {
+    protected TableDescriptor getDescriptor() {
       return desc;
     }
 
@@ -736,36 +650,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] deleteTables(String regex) throws IOException {
-    return deleteTables(Pattern.compile(regex));
-  }
-
-  /**
-   * Delete tables matching the passed in pattern and wait on completion.
-   *
-   * Warning: Use this method carefully, there is no prompting and the effect is
-   * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
-   * {@link #deleteTable(TableName)}
-   *
-   * @param pattern The pattern to match table names against
-   * @return Table descriptors for tables that couldn't be deleted
-   * @throws IOException
-   */
-  @Override
-  public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
-    List<HTableDescriptor> failed = new LinkedList<>();
-    for (HTableDescriptor table : listTables(pattern)) {
-      try {
-        deleteTable(table.getTableName());
-      } catch (IOException ex) {
-        LOG.info("Failed to delete table " + table.getTableName(), ex);
-        failed.add(table);
-      }
-    }
-    return failed.toArray(new HTableDescriptor[failed.size()]);
-  }
-
-  @Override
   public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
       throws IOException {
     TruncateTableResponse response =
@@ -864,27 +748,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] enableTables(String regex) throws IOException {
-    return enableTables(Pattern.compile(regex));
-  }
-
-  @Override
-  public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
-    List<HTableDescriptor> failed = new LinkedList<>();
-    for (HTableDescriptor table : listTables(pattern)) {
-      if (isTableDisabled(table.getTableName())) {
-        try {
-          enableTable(table.getTableName());
-        } catch (IOException ex) {
-          LOG.info("Failed to enable table " + table.getTableName(), ex);
-          failed.add(table);
-        }
-      }
-    }
-    return failed.toArray(new HTableDescriptor[failed.size()]);
-  }
-
-  @Override
   public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     DisableTableResponse response = executeCallable(
@@ -924,27 +787,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] disableTables(String regex) throws IOException {
-    return disableTables(Pattern.compile(regex));
-  }
-
-  @Override
-  public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
-    List<HTableDescriptor> failed = new LinkedList<>();
-    for (HTableDescriptor table : listTables(pattern)) {
-      if (isTableEnabled(table.getTableName())) {
-        try {
-          disableTable(table.getTableName());
-        } catch (IOException ex) {
-          LOG.info("Failed to disable table " + table.getTableName(), ex);
-          failed.add(table);
-        }
-      }
-    }
-    return failed.toArray(new HTableDescriptor[failed.size()]);
-  }
-
-  @Override
   public boolean isTableEnabled(final TableName tableName) throws IOException {
     checkTableExists(tableName);
     return executeCallable(new RpcRetryingCallable<Boolean>() {
@@ -971,33 +813,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException {
-    return connection.isTableAvailable(tableName, splitKeys);
-  }
-
-  @Override
-  public Pair<Integer, Integer> getAlterStatus(final TableName tableName) throws IOException {
-    return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected Pair<Integer, Integer> rpcCall() throws Exception {
-        setPriority(tableName);
-        GetSchemaAlterStatusRequest req = RequestConverter
-            .buildGetSchemaAlterStatusRequest(tableName);
-        GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(getRpcController(), req);
-        Pair<Integer, Integer> pair = new Pair<>(ret.getYetToUpdateRegions(),
-            ret.getTotalRegions());
-        return pair;
-      }
-    });
-  }
-
-  @Override
-  public Pair<Integer, Integer> getAlterStatus(final byte[] tableName) throws IOException {
-    return getAlterStatus(TableName.valueOf(tableName));
-  }
-
-  @Override
   public Future<Void> addColumnFamilyAsync(final TableName tableName,
       final ColumnFamilyDescriptor columnFamily) throws IOException {
     AddColumnResponse response =
@@ -1029,18 +844,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * {@inheritDoc}
-   * @deprecated Since 2.0. Will be removed in 3.0. Use
-   *     {@link #deleteColumnFamily(TableName, byte[])} instead.
-   */
-  @Override
-  @Deprecated
-  public void deleteColumn(final TableName tableName, final byte[] columnFamily)
-      throws IOException {
-    deleteColumnFamily(tableName, columnFamily);
-  }
-
   @Override
   public Future<Void> deleteColumnFamilyAsync(final TableName tableName, final byte[] columnFamily)
       throws IOException {
@@ -1107,45 +910,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  @Deprecated
-  @Override
-  public void closeRegion(final String regionName, final String unused) throws IOException {
-    unassign(Bytes.toBytes(regionName), true);
-  }
-
-  @Deprecated
-  @Override
-  public void closeRegion(final byte [] regionName, final String unused) throws IOException {
-    unassign(regionName, true);
-  }
-
-  @Deprecated
-  @Override
-  public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
-      final String unused) throws IOException {
-    unassign(Bytes.toBytes(encodedRegionName), true);
-    return true;
-  }
-
-  @Deprecated
-  @Override
-  public void closeRegion(final ServerName unused, final HRegionInfo hri) throws IOException {
-    unassign(hri.getRegionName(), true);
-  }
-
-  /**
-   * @param sn
-   * @return List of {@link HRegionInfo}.
-   * @throws IOException
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             Use {@link #getRegions(ServerName)}.
-   */
-  @Deprecated
-  @Override
-  public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
-    return getRegions(sn).stream().map(ImmutableHRegionInfo::new).collect(Collectors.toList());
-  }
-
   @Override
   public void flush(final TableName tableName) throws IOException {
     checkTableExists(tableName);
@@ -1666,24 +1430,6 @@ public class HBaseAdmin implements Admin {
    * @param forcible true if do a compulsory merge, otherwise we will only merge
    *          two adjacent regions
    * @throws IOException
-   * @deprecated Since 2.0. Will be removed in 3.0. Use
-   *     {@link #mergeRegionsAsync(byte[], byte[], boolean)} instead.
-   */
-  @Deprecated
-  @Override
-  public void mergeRegions(final byte[] nameOfRegionA,
-      final byte[] nameOfRegionB, final boolean forcible)
-      throws IOException {
-    mergeRegionsAsync(nameOfRegionA, nameOfRegionB, forcible);
-  }
-
-  /**
-   * Merge two regions. Asynchronous operation.
-   * @param nameOfRegionA encoded or full name of region a
-   * @param nameOfRegionB encoded or full name of region b
-   * @param forcible true if do a compulsory merge, otherwise we will only merge
-   *          two adjacent regions
-   * @throws IOException
    */
   @Override
   public Future<Void> mergeRegionsAsync(
@@ -1710,9 +1456,9 @@ public class HBaseAdmin implements Admin {
     assert(nameofRegionsToMerge.length >= 2);
     byte[][] encodedNameofRegionsToMerge = new byte[nameofRegionsToMerge.length][];
     for(int i = 0; i < nameofRegionsToMerge.length; i++) {
-      encodedNameofRegionsToMerge[i] = HRegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ?
+      encodedNameofRegionsToMerge[i] = RegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ?
           nameofRegionsToMerge[i] :
-          Bytes.toBytes(HRegionInfo.encodeRegionName(nameofRegionsToMerge[i]));
+          Bytes.toBytes(RegionInfo.encodeRegionName(nameofRegionsToMerge[i]));
     }
 
     TableName tableName = null;
@@ -1722,7 +1468,7 @@ public class HBaseAdmin implements Admin {
       pair = getRegion(nameofRegionsToMerge[i]);
 
       if (pair != null) {
-        if (pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+        if (pair.getFirst().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
           throw new IllegalArgumentException ("Can't invoke merge on non-default regions directly");
         }
         if (tableName == null) {
@@ -1811,18 +1557,17 @@ public class HBaseAdmin implements Admin {
   @Override
   public Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint)
       throws IOException {
-    byte[] encodedNameofRegionToSplit = HRegionInfo.isEncodedRegionName(regionName) ?
-        regionName : Bytes.toBytes(HRegionInfo.encodeRegionName(regionName));
+    byte[] encodedNameofRegionToSplit = RegionInfo.isEncodedRegionName(regionName) ?
+        regionName : Bytes.toBytes(RegionInfo.encodeRegionName(regionName));
     Pair<RegionInfo, ServerName> pair = getRegion(regionName);
     if (pair != null) {
       if (pair.getFirst() != null &&
-          pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+          pair.getFirst().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
         throw new IllegalArgumentException ("Can't invoke split on non-default regions directly");
       }
     } else {
-      throw new UnknownRegionException (
-          "Can't invoke merge on unknown region "
-              + Bytes.toStringBinary(encodedNameofRegionToSplit));
+      throw new UnknownRegionException(
+        "Can't invoke split on unknown region " + Bytes.toStringBinary(encodedNameofRegionToSplit));
     }
 
     return splitRegionAsync(pair.getFirst(), splitPoint);
@@ -1877,11 +1622,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void splitRegion(final byte[] regionName) throws IOException {
-    splitRegion(regionName, null);
-  }
-
-  @Override
   public void split(final TableName tableName, final byte[] splitPoint) throws IOException {
     checkTableExists(tableName);
     for (HRegionLocation loc : connection.locateRegions(tableName, false, false)) {
@@ -1904,23 +1644,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  @Override
-  public void splitRegion(final byte[] regionName, final byte [] splitPoint) throws IOException {
-    Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
-    if (regionServerPair == null) {
-      throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
-    }
-    if (regionServerPair.getFirst() != null &&
-        regionServerPair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
-      throw new IllegalArgumentException("Can't split replicas directly. "
-          + "Replicas are auto-split when their primary is split.");
-    }
-    if (regionServerPair.getSecond() == null) {
-      throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
-    }
-    splitRegionAsync(regionServerPair.getFirst(), splitPoint);
-  }
-
   private static class ModifyTableFuture extends TableFuture<Void> {
     public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName,
         final ModifyTableResponse response) {
@@ -1936,16 +1659,6 @@ public class HBaseAdmin implements Admin {
     public String getOperationType() {
       return "MODIFY";
     }
-
-    @Override
-    protected Void postOperationResult(final Void result, final long deadlineTs)
-        throws IOException, TimeoutException {
-      // The modify operation on the table is asynchronous on the server side irrespective
-      // of whether Procedure V2 is supported or not. So, we wait in the client till
-      // all regions get updated.
-      waitForSchemaUpdate(deadlineTs);
-      return result;
-    }
   }
 
   /**
@@ -1978,9 +1691,9 @@ public class HBaseAdmin implements Admin {
           ServerName sn = null;
           if (rl != null) {
             for (HRegionLocation h : rl.getRegionLocations()) {
-              if (h != null && encodedName.equals(h.getRegionInfo().getEncodedName())) {
+              if (h != null && encodedName.equals(h.getRegion().getEncodedName())) {
                 sn = h.getServerName();
-                info = h.getRegionInfo();
+                info = h.getRegion();
                 matched = true;
               }
             }
@@ -2005,13 +1718,12 @@ public class HBaseAdmin implements Admin {
    * name, the input is returned as is. We don't throw unknown
    * region exception.
    */
-  private byte[] getRegionName(
-      final byte[] regionNameOrEncodedRegionName) throws IOException {
+  private byte[] getRegionName(final byte[] regionNameOrEncodedRegionName) throws IOException {
     if (Bytes.equals(regionNameOrEncodedRegionName,
-        HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
-          || Bytes.equals(regionNameOrEncodedRegionName,
-            HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
-      return HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
+      RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()) ||
+      Bytes.equals(regionNameOrEncodedRegionName,
+        RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
+      return RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName();
     }
     byte[] tmp = regionNameOrEncodedRegionName;
     Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
@@ -2273,25 +1985,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected HTableDescriptor[] rpcCall() throws Exception {
-        List<TableSchema> list =
-            master.listTableDescriptorsByNamespace(getRpcController(),
-                ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
-                .build()).getTableSchemaList();
-        HTableDescriptor[] res = new HTableDescriptor[list.size()];
-        for(int i=0; i < list.size(); i++) {
-          res[i] = new ImmutableHTableDescriptor(ProtobufUtil.toTableDescriptor(list.get(i)));
-        }
-        return res;
-      }
-    });
-  }
-
-  @Override
   public TableName[] listTableNamesByNamespace(final String name) throws IOException {
     return executeCallable(new MasterCallable<TableName[]>(getConnection(),
         getRpcControllerFactory()) {
@@ -2312,10 +2005,11 @@ public class HBaseAdmin implements Admin {
 
   /**
    * Is HBase available? Throw an exception if not.
+   * <p/>
+   * TODO: do not expose ZKConnectionException.
    * @param conf system configuration
    * @throws MasterNotRunningException if the master is not running.
-   * @throws ZooKeeperConnectionException if unable to connect to zookeeper. // TODO do not expose
-   *           ZKConnectionException.
+   * @throws ZooKeeperConnectionException if unable to connect to zookeeper.
    */
   public static void available(final Configuration conf)
       throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
@@ -2327,59 +2021,12 @@ public class HBaseAdmin implements Admin {
     // Check ZK first.
     // If the connection exists, we may have a connection to ZK that does not work anymore
     try (ClusterConnection connection =
-        (ClusterConnection) ConnectionFactory.createConnection(copyOfConf)) {
+      (ClusterConnection) ConnectionFactory.createConnection(copyOfConf)) {
       // can throw MasterNotRunningException
       connection.isMasterRunning();
     }
   }
 
-  /**
-   *
-   * @param tableName
-   * @return List of {@link HRegionInfo}.
-   * @throws IOException
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             Use {@link #getRegions(TableName)}.
-   */
-  @Deprecated
-  @Override
-  public List<HRegionInfo> getTableRegions(final TableName tableName)
-    throws IOException {
-    return getRegions(tableName).stream()
-        .map(ImmutableHRegionInfo::new)
-        .collect(Collectors.toList());
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-  }
-
-  @Override
-  public HTableDescriptor[] getTableDescriptorsByTableName(final List<TableName> tableNames)
-  throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected HTableDescriptor[] rpcCall() throws Exception {
-        GetTableDescriptorsRequest req =
-            RequestConverter.buildGetTableDescriptorsRequest(tableNames);
-        return ProtobufUtil
-            .toTableDescriptorList(master.getTableDescriptors(getRpcController(), req)).stream()
-            .map(ImmutableHTableDescriptor::new).toArray(HTableDescriptor[]::new);
-      }
-    });
-  }
-
-  @Override
-  public HTableDescriptor[] getTableDescriptors(List<String> names)
-  throws IOException {
-    List<TableName> tableNames = new ArrayList<>(names.size());
-    for(String name : names) {
-      tableNames.add(TableName.valueOf(name));
-    }
-    return getTableDescriptorsByTableName(tableNames);
-  }
-
   private RollWALWriterResponse rollWALWriterImpl(final ServerName sn) throws IOException,
       FailedLogCloseException {
     final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
@@ -2540,12 +2187,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void restoreSnapshot(final byte[] snapshotName)
-      throws IOException, RestoreSnapshotException {
-    restoreSnapshot(Bytes.toString(snapshotName));
-  }
-
-  @Override
   public void restoreSnapshot(final String snapshotName)
       throws IOException, RestoreSnapshotException {
     boolean takeFailSafeSnapshot =
@@ -2554,12 +2195,6 @@ public class HBaseAdmin implements Admin {
     restoreSnapshot(snapshotName, takeFailSafeSnapshot);
   }
 
-  @Override
-  public void restoreSnapshot(final byte[] snapshotName, final boolean takeFailSafeSnapshot)
-      throws IOException, RestoreSnapshotException {
-    restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
-  }
-
   /**
    * Check whether the snapshot exists and contains disabled table
    *
@@ -2660,24 +2295,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public Future<Void> restoreSnapshotAsync(final String snapshotName)
-      throws IOException, RestoreSnapshotException {
-    TableName tableName = getTableNameBeforeRestoreSnapshot(snapshotName);
-
-    // The table does not exists, switch to clone.
-    if (!tableExists(tableName)) {
-      return cloneSnapshotAsync(snapshotName, tableName);
-    }
-
-    // Check if the table is disabled
-    if (!isTableDisabled(tableName)) {
-      throw new TableNotDisabledException(tableName);
-    }
-
-    return internalRestoreSnapshotAsync(snapshotName, tableName, false);
-  }
-
-  @Override
   public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
       boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException {
     if (tableExists(tableName)) {
@@ -2847,11 +2464,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public List<SnapshotDescription> listSnapshots(String regex) throws IOException {
-    return listSnapshots(Pattern.compile(regex));
-  }
-
-  @Override
   public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
     List<SnapshotDescription> matched = new LinkedList<>();
     List<SnapshotDescription> snapshots = listSnapshots();
@@ -2864,12 +2476,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
-      String snapshotNameRegex) throws IOException {
-    return listTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
-  }
-
-  @Override
   public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
       Pattern snapshotNamePattern) throws IOException {
     TableName[] tableNames = listTableNames(tableNamePattern);
@@ -2910,11 +2516,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void deleteSnapshots(final String regex) throws IOException {
-    deleteSnapshots(Pattern.compile(regex));
-  }
-
-  @Override
   public void deleteSnapshots(final Pattern pattern) throws IOException {
     List<SnapshotDescription> snapshots = listSnapshots(pattern);
     for (final SnapshotDescription snapshot : snapshots) {
@@ -2939,12 +2540,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex)
-      throws IOException {
-    deleteTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
-  }
-
-  @Override
   public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
       throws IOException {
     List<SnapshotDescription> snapshots = listTableSnapshots(tableNamePattern, snapshotNamePattern);
@@ -2970,11 +2565,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public QuotaRetriever getQuotaRetriever(final QuotaFilter filter) throws IOException {
-    return QuotaRetriever.open(conf, filter);
-  }
-
-  @Override
   public List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException {
     List<QuotaSettings> quotas = new LinkedList<>();
     try (QuotaRetriever retriever = QuotaRetriever.open(conf, filter)) {
@@ -3035,21 +2625,6 @@ public class HBaseAdmin implements Admin {
     };
   }
 
-  /**
-   * Simple {@link Abortable}, throwing RuntimeException on abort.
-   */
-  private static class ThrowableAbortable implements Abortable {
-    @Override
-    public void abort(String why, Throwable e) {
-      throw new RuntimeException(why, e);
-    }
-
-    @Override
-    public boolean isAborted() {
-      return true;
-    }
-  }
-
   @Override
   public CoprocessorRpcChannel coprocessorService(final ServerName serverName) {
     return new SyncCoprocessorRpcChannel() {
@@ -3560,7 +3135,7 @@ public class HBaseAdmin implements Admin {
     /**
      * @return the table descriptor
      */
-    protected TableDescriptor getTableDescriptor() throws IOException {
+    protected TableDescriptor getDescriptor() throws IOException {
       return getAdmin().getDescriptor(getTableName());
     }
 
@@ -3642,19 +3217,9 @@ public class HBaseAdmin implements Admin {
       });
     }
 
-    protected void waitForSchemaUpdate(final long deadlineTs)
-        throws IOException, TimeoutException {
-      waitForState(deadlineTs, new TableWaitForStateCallable() {
-        @Override
-        public boolean checkState(int tries) throws IOException {
-          return getAdmin().getAlterStatus(tableName).getFirst() == 0;
-        }
-      });
-    }
-
     protected void waitForAllRegionsOnline(final long deadlineTs, final byte[][] splitKeys)
         throws IOException, TimeoutException {
-      final TableDescriptor desc = getTableDescriptor();
+      final TableDescriptor desc = getDescriptor();
       final AtomicInteger actualRegCount = new AtomicInteger(0);
       final MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
         @Override
@@ -3668,16 +3233,18 @@ public class HBaseAdmin implements Admin {
           if (l == null) {
             return true;
           }
-          if (!l.getRegionInfo().getTable().equals(desc.getTableName())) {
+          if (!l.getRegion().getTable().equals(desc.getTableName())) {
             return false;
           }
-          if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
+          if (l.getRegion().isOffline() || l.getRegion().isSplit()) {
+            return true;
+          }
           HRegionLocation[] locations = list.getRegionLocations();
           for (HRegionLocation location : locations) {
             if (location == null) continue;
             ServerName serverName = location.getServerName();
             // Make sure that regions are assigned to server
-            if (serverName != null && serverName.getHostAndPort() != null) {
+            if (serverName != null && serverName.getAddress() != null) {
               actualRegCount.incrementAndGet();
             }
           }
@@ -4132,7 +3699,7 @@ public class HBaseAdmin implements Admin {
       @Override
       protected List<ServerName> rpcCall() throws Exception {
         ClearDeadServersRequest req = RequestConverter.
-          buildClearDeadServersRequest(servers == null? Collections.EMPTY_LIST: servers);
+          buildClearDeadServersRequest(servers == null? Collections.emptyList(): servers);
         return ProtobufUtil.toServerNameList(
                 master.clearDeadServers(getRpcController(), req).getServerNameList());
       }
@@ -4146,7 +3713,7 @@ public class HBaseAdmin implements Admin {
     if (tableExists(newTableName)) {
       throw new TableExistsException(newTableName);
     }
-    TableDescriptor htd = TableDescriptorBuilder.copy(newTableName, getTableDescriptor(tableName));
+    TableDescriptor htd = TableDescriptorBuilder.copy(newTableName, getDescriptor(tableName));
     if (preserveSplits) {
       createTable(htd, getTableSplits(tableName));
     } else {
@@ -4296,4 +3863,8 @@ public class HBaseAdmin implements Admin {
       }
     });
   }
+
+  @Override
+  public void close() {
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 9b3afd9..23e7847 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
+
 // DO NOT MAKE USE OF THESE IMPORTS! THEY ARE HERE FOR COPROCESSOR ENDPOINTS ONLY.
 // Internally, we use shaded protobuf. This below are part of our public API.
 //SEE ABOVE NOTE!
@@ -25,26 +27,44 @@ import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
 import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
-
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.io.TimeRange;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
@@ -54,27 +74,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequ
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
-import org.apache.hadoop.hbase.util.Threads;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
 
 /**
  * An implementation of {@link Table}. Used to communicate with a single HBase table.
@@ -224,17 +223,6 @@ public class HTable implements Table {
   }
 
   @Override
-  @Deprecated
-  public HTableDescriptor getTableDescriptor() throws IOException {
-    HTableDescriptor htd = HBaseAdmin.getHTableDescriptor(tableName, connection, rpcCallerFactory,
-      rpcControllerFactory, operationTimeoutMs, readRpcTimeoutMs);
-    if (htd != null) {
-      return new ImmutableHTableDescriptor(htd);
-    }
-    return null;
-  }
-
-  @Override
   public TableDescriptor getDescriptor() throws IOException {
     return HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory,
       rpcControllerFactory, operationTimeoutMs, readRpcTimeoutMs);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
index 05876f0..1b1fe43 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
@@ -21,23 +21,22 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.regex.Pattern;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.BlockingInterface;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Utility client for doing access control admin operations.
@@ -254,7 +253,7 @@ public class AccessControlClient {
         CoprocessorRpcChannel service = table.coprocessorService(HConstants.EMPTY_START_ROW);
         BlockingInterface protocol =
             AccessControlProtos.AccessControlService.newBlockingStub(service);
-        HTableDescriptor[] htds = null;
+        List<TableDescriptor> htds = null;
         if (tableRegex == null || tableRegex.isEmpty()) {
           permList = AccessControlUtil.getUserPermissions(null, protocol, userName);
         } else if (tableRegex.charAt(0) == '@') { // Namespaces
@@ -268,8 +267,8 @@ public class AccessControlClient {
             }
           }
         } else { // Tables
-          htds = admin.listTables(Pattern.compile(tableRegex), true);
-          for (HTableDescriptor htd : htds) {
+          htds = admin.listTableDescriptors(Pattern.compile(tableRegex), true);
+          for (TableDescriptor htd : htds) {
             permList.addAll(AccessControlUtil.getUserPermissions(null, protocol, htd.getTableName(),
               null, null, userName));
           }
@@ -352,9 +351,9 @@ public class AccessControlClient {
         CoprocessorRpcChannel service = table.coprocessorService(HConstants.EMPTY_START_ROW);
         BlockingInterface protocol =
             AccessControlProtos.AccessControlService.newBlockingStub(service);
-        HTableDescriptor[] htds = admin.listTables(Pattern.compile(tableRegex), true);
+        List<TableDescriptor> htds = admin.listTableDescriptors(Pattern.compile(tableRegex), true);
         // Retrieve table permissions
-        for (HTableDescriptor htd : htds) {
+        for (TableDescriptor htd : htds) {
           permList.addAll(AccessControlUtil.getUserPermissions(null, protocol, htd.getTableName(),
             columnFamily, columnQualifier, userName));
         }
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
index c43bda6..c0c7fbd 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
@@ -139,7 +139,7 @@ public final class FutureUtils {
   }
 
   /**
-   * A helper class for getting the result of a Future, and convert the error to an
+   * A helper class for getting the result of a Future with timeout, and convert the error to an
    * {@link IOException}.
    */
   public static <T> T get(Future<T> future, long timeout, TimeUnit unit) throws IOException {
diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java
index 3bec203..b25c17d 100644
--- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java
+++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java
@@ -151,7 +151,7 @@ public class TestCoprocessorTableEndpoint {
   private static void updateTable(HTableDescriptor desc) throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
     admin.disableTable(desc.getTableName());
-    admin.modifyTable(desc.getTableName(), desc);
+    admin.modifyTable(desc);
     admin.enableTable(desc.getTableName());
   }
 
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
index 30a3db9..5a5d6d0 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
@@ -522,11 +522,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
     // and the restore put the cluster back to Initial configuration, HAdmin instance will need
     // to refresh its connections (otherwise it will return incorrect information) or we can
     // point it to new instance.
-    try {
-      admin.close();
-    } catch (IOException ioe) {
-      LOG.warn("While closing the old connection", ioe);
-    }
+    admin.close();
     this.admin = this.connection.getAdmin();
     LOG.info("Added new HBaseAdmin");
     return true;
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
index 2fb12c3..61d0e7e 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
-
+import java.util.regex.Pattern;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
@@ -151,8 +152,10 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
   public void cleanUpCluster() throws Exception {
     if (!keepObjectsAtTheEnd) {
       Admin admin = util.getAdmin();
-      admin.disableTables("ittable-\\d+");
-      admin.deleteTables("ittable-\\d+");
+      for (TableName tableName: admin.listTableNames(Pattern.compile("ittable-\\d+"))) {
+        admin.disableTable(tableName);
+        admin.deleteTable(tableName);
+      }
       NamespaceDescriptor [] nsds = admin.listNamespaceDescriptors();
       for(NamespaceDescriptor nsd: nsds) {
         if(nsd.getName().matches("itnamespace\\d+")) {
@@ -713,7 +716,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
         admin.modifyTable(td);
 
         // assertion
-        TableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
+        TableDescriptor freshTableDesc = admin.getDescriptor(tableName);
         ColumnFamilyDescriptor freshColumnDesc = freshTableDesc.getColumnFamily(columnDesc.getName());
         Assert.assertEquals("Encoding of column family: " + columnDesc + " was not altered",
             freshColumnDesc.getDataBlockEncoding().getId(), id);
@@ -781,7 +784,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
       Admin admin = connection.getAdmin();
       TableName tableName = selected.getTableName();
       try (Table table = connection.getTable(tableName)){
-        ArrayList<HRegionInfo> regionInfos = new ArrayList<>(admin.getTableRegions(
+        ArrayList<RegionInfo> regionInfos = new ArrayList<>(admin.getRegions(
             selected.getTableName()));
         int numRegions = regionInfos.size();
         // average number of rows to be added per action to each region
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
index dec1acd..1bd91fe 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
@@ -120,7 +120,7 @@ public class IntegrationTestIngestWithMOB extends IntegrationTestIngest {
     TableName tableName = getTablename();
     try (Connection connection = ConnectionFactory.createConnection();
          Admin admin = connection.getAdmin()) {
-      HTableDescriptor tableDesc = admin.getTableDescriptor(tableName);
+      HTableDescriptor tableDesc = new HTableDescriptor(admin.getDescriptor(tableName));
       LOG.info("Disabling table " + getTablename());
       admin.disableTable(tableName);
       ColumnFamilyDescriptor mobColumn = tableDesc.getColumnFamily(mobColumnFamily);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
index d1f466e..4c46926 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
@@ -278,7 +278,7 @@ public class IntegrationTestRegionReplicaPerf extends IntegrationTestBase {
     // one last sanity check, then send in the clowns!
     assertEquals("Table must be created with DisabledRegionSplitPolicy. Broken test.",
         DisabledRegionSplitPolicy.class.getName(),
-        util.getAdmin().getTableDescriptor(tableName).getRegionSplitPolicyClassName());
+        util.getAdmin().getDescriptor(tableName).getRegionSplitPolicyClassName());
     startMonkey();
 
     // collect a baseline without region replicas.
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
index 6db6da0..e90a269 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
@@ -261,7 +261,7 @@ public class Action {
     Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin();
     boolean result = false;
     try {
-      result = admin.balancer();
+      result = admin.balance();
     } catch (Exception e) {
       LOG.warn("Got exception while doing balance ", e);
     }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java
index f3fe6ff..5489668 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java
@@ -19,13 +19,12 @@
 package org.apache.hadoop.hbase.chaos.actions;
 
 import java.util.List;
-
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 
 /**
  * Region that queues a compaction of a random region from the table.
@@ -55,14 +54,14 @@ public class CompactRandomRegionOfTableAction extends Action {
 
     LOG.info("Performing action: Compact random region of table "
       + tableName + ", major=" + major);
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     if (regions == null || regions.isEmpty()) {
       LOG.info("Table " + tableName + " doesn't have regions to compact");
       return;
     }
 
-    HRegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
-      regions.toArray(new HRegionInfo[regions.size()]));
+    RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
+      regions.toArray(new RegionInfo[regions.size()]));
 
     try {
       if (major) {
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java
index c5a1156..be15a12 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java
@@ -19,12 +19,11 @@
 package org.apache.hadoop.hbase.chaos.actions;
 
 import java.util.List;
-
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 
 /**
 * Action that tries to flush a random region of a table.
@@ -48,14 +47,14 @@ public class FlushRandomRegionOfTableAction extends Action {
     Admin admin = util.getAdmin();
 
     LOG.info("Performing action: Flush random region of table " + tableName);
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     if (regions == null || regions.isEmpty()) {
       LOG.info("Table " + tableName + " doesn't have regions to flush");
       return;
     }
 
-    HRegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
-      regions.toArray(new HRegionInfo[regions.size()]));
+    RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
+      regions.toArray(new RegionInfo[regions.size()]));
     LOG.debug("Flushing region " + region.getRegionNameAsString());
     try {
       admin.flushRegion(region.getRegionName());
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java
index 686e622..9a5b057 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java
@@ -19,12 +19,11 @@
 package org.apache.hadoop.hbase.chaos.actions;
 
 import java.util.List;
-
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 
 /**
  * Action to merge regions of a table.
@@ -48,15 +47,15 @@ public class MergeRandomAdjacentRegionsOfTableAction extends Action {
     Admin admin = util.getAdmin();
 
     LOG.info("Performing action: Merge random adjacent regions of table " + tableName);
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     if (regions == null || regions.size() < 2) {
       LOG.info("Table " + tableName + " doesn't have enough regions to merge");
       return;
     }
 
     int i = RandomUtils.nextInt(0, regions.size() - 1);
-    HRegionInfo a = regions.get(i++);
-    HRegionInfo b = regions.get(i);
+    RegionInfo a = regions.get(i++);
+    RegionInfo b = regions.get(i);
     LOG.debug("Merging " + a.getRegionNameAsString() + " and " + b.getRegionNameAsString());
 
     // Don't try the merge if we're stopping
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
index 3496b9a..4c868a8 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
@@ -25,7 +25,6 @@ import java.util.EnumSet;
 import java.util.List;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants;
@@ -61,7 +60,7 @@ public class MoveRegionsOfTableAction extends Action {
     ServerName[] servers = getServers(admin);
 
     LOG.info("Performing action: Move regions of table {}", tableName);
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     if (regions == null || regions.isEmpty()) {
       LOG.info("Table {} doesn't have regions to move", tableName);
       return;
@@ -70,8 +69,7 @@ public class MoveRegionsOfTableAction extends Action {
     Collections.shuffle(regions);
 
     long start = System.currentTimeMillis();
-    for (HRegionInfo regionInfo:regions) {
-
+    for (RegionInfo regionInfo : regions) {
       // Don't try the move if we're stopping
       if (context.isStopping()) {
         return;
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java
index de266f5..890ff92 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java
@@ -19,12 +19,11 @@
 package org.apache.hadoop.hbase.chaos.actions;
 
 import java.util.List;
-
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 
 /**
 * Action that tries to split a random region of a table.
@@ -48,7 +47,7 @@ public class SplitRandomRegionOfTableAction extends Action {
     Admin admin = util.getAdmin();
 
     LOG.info("Performing action: Split random region of table " + tableName);
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     if (regions == null || regions.isEmpty()) {
       LOG.info("Table " + tableName + " doesn't have regions to split");
       return;
@@ -58,11 +57,11 @@ public class SplitRandomRegionOfTableAction extends Action {
       return;
     }
 
-    HRegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
-        regions.toArray(new HRegionInfo[regions.size()]));
+    RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
+        regions.toArray(new RegionInfo[regions.size()]));
     LOG.debug("Splitting region " + region.getRegionNameAsString());
     try {
-      admin.splitRegion(region.getRegionName());
+      admin.splitRegionAsync(region.getRegionName(), null).get();
     } catch (Exception ex) {
       LOG.warn("Split failed, might be caused by other chaos: " + ex.getMessage());
     }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
index ba2e214..a28c9f6 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
@@ -670,9 +670,9 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase {
       LOG.error("Failure in chain verification: " + msg);
       try (Connection connection = ConnectionFactory.createConnection(context.getConfiguration());
           Admin admin = connection.getAdmin()) {
-        LOG.error("cluster status:\n" + admin.getClusterStatus());
+        LOG.error("cluster metrics:\n" + admin.getClusterMetrics());
         LOG.error("table regions:\n"
-            + Joiner.on("\n").join(admin.getTableRegions(table)));
+            + Joiner.on("\n").join(admin.getRegions(table)));
       }
     }
   }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
index 4c2e379..e1ff49c 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
@@ -30,7 +30,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.NamespaceExistException;
@@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
 import org.apache.hadoop.hbase.ipc.FatalConnectionException;
 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
-import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -74,6 +73,8 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
+
 /**
  * Integration test that should benchmark how fast HBase can recover from failures. This test starts
  * different threads:
@@ -572,7 +573,7 @@ public class IntegrationTestMTTR {
       Admin admin = null;
       try {
         admin = util.getAdmin();
-        ClusterStatus status = admin.getClusterStatus();
+        ClusterMetrics status = admin.getClusterMetrics();
         return status != null;
       } finally {
         if (admin != null) {
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/rsgroup/IntegrationTestRSGroup.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/rsgroup/IntegrationTestRSGroup.java
index f5f2ff9..4f8b3ad 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/rsgroup/IntegrationTestRSGroup.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/rsgroup/IntegrationTestRSGroup.java
@@ -70,7 +70,7 @@ public class IntegrationTestRSGroup extends TestRSGroupsBase {
     deleteTableIfNecessary();
     deleteNamespaceIfNecessary();
     deleteGroups();
-    admin.setBalancerRunning(true, true);
+    admin.balancerSwitch(true, true);
 
     LOG.info("Restoring the cluster");
     ((IntegrationTestingUtility)TEST_UTIL).restoreCluster();
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 85897e9..2dae0e8 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
@@ -340,7 +340,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
         "Must specify an existing table for read commands. Run a write command first.");
     }
     HTableDescriptor desc =
-      exists ? admin.getTableDescriptor(TableName.valueOf(opts.tableName)) : null;
+      exists ? new HTableDescriptor(admin.getDescriptor(TableName.valueOf(opts.tableName))) : null;
     byte[][] splits = getSplits(opts);
 
     // recreate the table when user has requested presplit or when existing
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index e8bb805..f4f8236 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -700,8 +700,8 @@ public class TestHFileOutputFormat2  {
       // Perform the actual load
       for (HFileOutputFormat2.TableInfo singleTableInfo : tableInfo) {
         Path tableDir = testDir;
-        String tableNameStr = singleTableInfo.getHTableDescriptor().getNameAsString();
-        LOG.info("Running LoadIncrementalHFiles on table" + tableNameStr);
+        String tableNameStr = singleTableInfo.getTableDescriptor().getTableName().getNameAsString();
+        LOG.info("Running BulkLoadHFiles on table" + tableNameStr);
         if (writeMultipleTables) {
           tableDir = new Path(testDir, tableNameStr);
         }
@@ -1237,7 +1237,7 @@ public class TestHFileOutputFormat2  {
       // deep inspection: get the StoreFile dir
       final Path storePath = new Path(
         FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]),
-          new Path(admin.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
+          new Path(admin.getRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
             Bytes.toString(FAMILIES[0])));
       assertEquals(0, fs.listStatus(storePath).length);
 
@@ -1318,7 +1318,7 @@ public class TestHFileOutputFormat2  {
       // deep inspection: get the StoreFile dir
       final Path storePath = new Path(
         FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]),
-          new Path(admin.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
+          new Path(admin.getRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
             Bytes.toString(FAMILIES[0])));
       assertEquals(0, fs.listStatus(storePath).length);
 
@@ -1411,8 +1411,10 @@ public class TestHFileOutputFormat2  {
           Admin admin = c.getAdmin();
           RegionLocator regionLocator = c.getRegionLocator(tname)) {
         Path outDir = new Path("incremental-out");
-        runIncrementalPELoad(conf, Arrays.asList(new HFileOutputFormat2.TableInfo(admin
-                .getTableDescriptor(tname), regionLocator)), outDir, false);
+        runIncrementalPELoad(conf,
+          Arrays
+            .asList(new HFileOutputFormat2.TableInfo(admin.getDescriptor(tname), regionLocator)),
+          outDir, false);
       }
     } else {
       throw new RuntimeException(
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
index c988854..d47d60b 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
@@ -79,8 +79,8 @@ public class TestExportSnapshot {
   public final TestName testName = new TestName();
 
   protected TableName tableName;
-  private byte[] emptySnapshotName;
-  private byte[] snapshotName;
+  private String emptySnapshotName;
+  private String snapshotName;
   private int tableNumFiles;
   private Admin admin;
 
@@ -117,8 +117,8 @@ public class TestExportSnapshot {
     this.admin = TEST_UTIL.getAdmin();
 
     tableName = TableName.valueOf("testtb-" + testName.getMethodName());
-    snapshotName = Bytes.toBytes("snaptb0-" + testName.getMethodName());
-    emptySnapshotName = Bytes.toBytes("emptySnaptb0-" + testName.getMethodName());
+    snapshotName = "snaptb0-" + testName.getMethodName();
+    emptySnapshotName = "emptySnaptb0-" + testName.getMethodName();
 
     // create Table
     createTable();
@@ -128,7 +128,7 @@ public class TestExportSnapshot {
 
     // Add some rows
     SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 50, FAMILY);
-    tableNumFiles = admin.getTableRegions(tableName).size();
+    tableNumFiles = admin.getRegions(tableName).size();
 
     // take a snapshot
     admin.snapshot(snapshotName, tableName);
@@ -186,18 +186,18 @@ public class TestExportSnapshot {
 
   @Test
   public void testExportWithTargetName() throws Exception {
-    final byte[] targetName = Bytes.toBytes("testExportWithTargetName");
+    final String targetName = "testExportWithTargetName";
     testExportFileSystemState(tableName, snapshotName, targetName, tableNumFiles);
   }
 
-  private void testExportFileSystemState(final TableName tableName, final byte[] snapshotName,
-      final byte[] targetName, int filesExpected) throws Exception {
+  private void testExportFileSystemState(final TableName tableName, final String snapshotName,
+      final String targetName, int filesExpected) throws Exception {
     testExportFileSystemState(tableName, snapshotName, targetName,
       filesExpected, getHdfsDestinationDir(), false);
   }
 
   protected void testExportFileSystemState(final TableName tableName,
-      final byte[] snapshotName, final byte[] targetName, int filesExpected,
+      final String snapshotName, final String targetName, int filesExpected,
       Path copyDir, boolean overwrite) throws Exception {
     testExportFileSystemState(TEST_UTIL.getConfiguration(), tableName, snapshotName, targetName,
       filesExpected, TEST_UTIL.getDefaultRootDirPath(), copyDir,
@@ -208,7 +208,7 @@ public class TestExportSnapshot {
    * Creates destination directory, runs ExportSnapshot() tool, and runs some verifications.
    */
   protected static void testExportFileSystemState(final Configuration conf, final TableName tableName,
-      final byte[] snapshotName, final byte[] targetName, final int filesExpected,
+      final String snapshotName, final String targetName, final int filesExpected,
       final Path sourceDir, Path copyDir, final boolean overwrite,
       final RegionPredicate bypassregionPredicate, boolean success) throws Exception {
     URI hdfsUri = FileSystem.get(conf).getUri();
@@ -217,12 +217,12 @@ public class TestExportSnapshot {
 
     List<String> opts = new ArrayList<>();
     opts.add("--snapshot");
-    opts.add(Bytes.toString(snapshotName));
+    opts.add(snapshotName);
     opts.add("--copy-to");
     opts.add(copyDir.toString());
-    if (targetName != snapshotName) {
+    if (!targetName.equals(snapshotName)) {
       opts.add("--target");
-      opts.add(Bytes.toString(targetName));
+      opts.add(targetName);
     }
     if (overwrite) opts.add("--overwrite");
 
@@ -230,7 +230,7 @@ public class TestExportSnapshot {
     int res = run(conf, new ExportSnapshot(), opts.toArray(new String[opts.size()]));
     assertEquals(success ? 0 : 1, res);
     if (!success) {
-      final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(targetName));
+      final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, targetName);
       assertFalse(fs.exists(new Path(copyDir, targetDir)));
       return;
     }
@@ -247,12 +247,12 @@ public class TestExportSnapshot {
 
     // compare the snapshot metadata and verify the hfiles
     final FileSystem hdfs = FileSystem.get(hdfsUri, conf);
-    final Path snapshotDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(snapshotName));
-    final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(targetName));
+    final Path snapshotDir = new Path(HConstants.SNAPSHOT_DIR_NAME, snapshotName);
+    final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, targetName);
     verifySnapshotDir(hdfs, new Path(sourceDir, snapshotDir),
         fs, new Path(copyDir, targetDir));
     Set<String> snapshotFiles = verifySnapshot(conf, fs, copyDir, tableName,
-      Bytes.toString(targetName), bypassregionPredicate);
+      targetName, bypassregionPredicate);
     assertEquals(filesExpected, snapshotFiles.size());
   }
 
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
index 0720b36..8da5805 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock;
 import org.apache.hadoop.hbase.testclassification.MapReduceTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -98,7 +97,7 @@ public class TestExportSnapshotNoCluster {
     builder.commit();
     int snapshotFilesCount = r1Files.length + r2Files.length;
 
-    byte[] snapshotName = Bytes.toBytes(builder.getSnapshotDescription().getName());
+    String snapshotName = builder.getSnapshotDescription().getName();
     TableName tableName = builder.getTableDescriptor().getTableName();
     TestExportSnapshot.testExportFileSystemState(TEST_UTIL.getConfiguration(),
       tableName, snapshotName, snapshotName, snapshotFilesCount,
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
index 3ff25f9..6533de1 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase.rest;
 
 import java.io.IOException;
-
+import java.util.List;
 import javax.servlet.ServletContext;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
@@ -33,16 +33,16 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.UriInfo;
-
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.rest.model.NamespacesInstanceModel;
 import org.apache.hadoop.hbase.rest.model.TableListModel;
 import org.apache.hadoop.hbase.rest.model.TableModel;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Implements the following REST end points:
@@ -100,9 +100,10 @@ public class NamespacesInstanceResource extends ResourceBase {
     if(queryTables){
       TableListModel tableModel = new TableListModel();
       try{
-        HTableDescriptor[] tables = servlet.getAdmin().listTableDescriptorsByNamespace(namespace);
-        for(int i = 0; i < tables.length; i++){
-          tableModel.add(new TableModel(tables[i].getTableName().getQualifierAsString()));
+        List<TableDescriptor> tables =
+          servlet.getAdmin().listTableDescriptorsByNamespace(Bytes.toBytes(namespace));
+        for (TableDescriptor table : tables) {
+          tableModel.add(new TableModel(table.getTableName().getQualifierAsString()));
         }
 
         servlet.getMetrics().incrementSucessfulGetRequests(1);
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
index 83a70ff..786fcb6 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
@@ -124,7 +124,7 @@ public class SchemaResource extends ResourceBase {
       }
       if (admin.tableExists(name)) {
         admin.disableTable(name);
-        admin.modifyTable(name, htd);
+        admin.modifyTable(htd);
         admin.enableTable(name);
         servlet.getMetrics().incrementSucessfulPutRequests(1);
       } else try {
@@ -152,7 +152,7 @@ public class SchemaResource extends ResourceBase {
         .build();
     }
     try {
-      HTableDescriptor htd = admin.getTableDescriptor(name);
+      HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(name));
       admin.disableTable(name);
       try {
         for (ColumnSchemaModel family: model.getColumns()) {
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
index 21d25e2..c5a125f 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
@@ -524,7 +524,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
    * @throws IOException
    */
   private boolean checkTable(RemoteAdmin admin) throws IOException {
-    HTableDescriptor tableDescriptor = getTableDescriptor();
+    HTableDescriptor tableDescriptor = getDescriptor();
     if (this.presplitRegions > 0) {
       // presplit requested
       if (admin.isTableAvailable(tableDescriptor.getTableName().getName())) {
@@ -548,7 +548,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     return tableExists;
   }
 
-  protected HTableDescriptor getTableDescriptor() {
+  protected HTableDescriptor getDescriptor() {
     if (TABLE_DESCRIPTOR == null) {
       TABLE_DESCRIPTOR = new HTableDescriptor(tableName);
       HColumnDescriptor family = new HColumnDescriptor(FAMILY_NAME);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BulkLoadObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BulkLoadObserver.java
index b69a727..094a7d9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BulkLoadObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BulkLoadObserver.java
@@ -51,7 +51,7 @@ public interface BulkLoadObserver {
       * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
       * If you need to get the region or table name, get it from the
       * <code>ctx</code> as follows: <code>code>ctx.getEnvironment().getRegion()</code>. Use
-      * getRegionInfo to fetch the encodedName and use getTableDescriptor() to get the tableName.
+      * getRegionInfo to fetch the encodedName and use getDescriptor() to get the tableName.
       * @param ctx the environment to interact with the framework and master
       */
     default void prePrepareBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx)
@@ -62,7 +62,7 @@ public interface BulkLoadObserver {
       * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
       * If you need to get the region or table name, get it from the
       * <code>ctx</code> as follows: <code>code>ctx.getEnvironment().getRegion()</code>. Use
-      * getRegionInfo to fetch the encodedName and use getTableDescriptor() to get the tableName.
+      * getRegionInfo to fetch the encodedName and use getDescriptor() to get the tableName.
       * @param ctx the environment to interact with the framework and master
       */
     default void preCleanupBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
index 73fabf8..283ebc2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
@@ -202,7 +202,7 @@ public class CatalogJanitor extends ScheduledChore {
     FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
     Path rootdir = this.services.getMasterFileSystem().getRootDir();
     Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
-    TableDescriptor htd = getTableDescriptor(mergedRegion.getTable());
+    TableDescriptor htd = getDescriptor(mergedRegion.getTable());
     HRegionFileSystem regionFs = null;
     try {
       regionFs = HRegionFileSystem.openRegionFromFileSystem(
@@ -410,7 +410,7 @@ public class CatalogJanitor extends ScheduledChore {
     }
 
     boolean references = false;
-    TableDescriptor parentDescriptor = getTableDescriptor(parent.getTable());
+    TableDescriptor parentDescriptor = getDescriptor(parent.getTable());
     try {
       regionFs = HRegionFileSystem.openRegionFromFileSystem(
           this.services.getConfiguration(), fs, tabledir, daughter, true);
@@ -428,7 +428,7 @@ public class CatalogJanitor extends ScheduledChore {
     return new Pair<>(Boolean.TRUE, Boolean.valueOf(references));
   }
 
-  private TableDescriptor getTableDescriptor(final TableName tableName)
+  private TableDescriptor getDescriptor(final TableName tableName)
       throws FileNotFoundException, IOException {
     return this.services.getTableDescriptors().get(tableName);
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
index ce4bc38..373f6d9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
@@ -229,7 +229,7 @@ public class RegionStateStore {
   // ============================================================================================
   public void splitRegion(RegionInfo parent, RegionInfo hriA, RegionInfo hriB,
       ServerName serverName) throws IOException {
-    TableDescriptor htd = getTableDescriptor(parent.getTable());
+    TableDescriptor htd = getDescriptor(parent.getTable());
     long parentOpenSeqNum = HConstants.NO_SEQNUM;
     if (htd.hasGlobalReplicationScope()) {
       parentOpenSeqNum = getOpenSeqNumForParentRegion(parent);
@@ -243,7 +243,7 @@ public class RegionStateStore {
   // ============================================================================================
   public void mergeRegions(RegionInfo child, RegionInfo hriA, RegionInfo hriB,
       ServerName serverName) throws IOException {
-    TableDescriptor htd = getTableDescriptor(child.getTable());
+    TableDescriptor htd = getDescriptor(child.getTable());
     long regionAOpenSeqNum = -1L;
     long regionBOpenSeqNum = -1L;
     if (htd.hasGlobalReplicationScope()) {
@@ -269,7 +269,7 @@ public class RegionStateStore {
   //  Table Descriptors helpers
   // ==========================================================================
   private boolean hasGlobalReplicationScope(TableName tableName) throws IOException {
-    return hasGlobalReplicationScope(getTableDescriptor(tableName));
+    return hasGlobalReplicationScope(getDescriptor(tableName));
   }
 
   private boolean hasGlobalReplicationScope(TableDescriptor htd) {
@@ -280,7 +280,7 @@ public class RegionStateStore {
     return htd != null ? htd.getRegionReplication() : 1;
   }
 
-  private TableDescriptor getTableDescriptor(TableName tableName) throws IOException {
+  private TableDescriptor getDescriptor(TableName tableName) throws IOException {
     return master.getTableDescriptors().get(tableName);
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
index fb7731f..b68c786 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
@@ -192,7 +192,7 @@ class RegionLocationFinder {
    */
   protected HDFSBlocksDistribution internalGetTopBlockLocation(RegionInfo region) {
     try {
-      TableDescriptor tableDescriptor = getTableDescriptor(region.getTable());
+      TableDescriptor tableDescriptor = getDescriptor(region.getTable());
       if (tableDescriptor != null) {
         HDFSBlocksDistribution blocksDistribution =
             HRegion.computeHDFSBlocksDistribution(getConf(), tableDescriptor, region);
@@ -213,7 +213,7 @@ class RegionLocationFinder {
    * @return TableDescriptor
    * @throws IOException
    */
-  protected TableDescriptor getTableDescriptor(TableName tableName) throws IOException {
+  protected TableDescriptor getDescriptor(TableName tableName) throws IOException {
     TableDescriptor tableDescriptor = null;
     try {
       if (this.services != null && this.services.getTableDescriptors() != null) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
index b5f8e82..3899342 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
@@ -18,9 +18,7 @@
  */
 package org.apache.hadoop.hbase.master.normalizer;
 
-import java.io.IOException;
 import java.util.Arrays;
-
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -78,8 +76,8 @@ public class SplitNormalizationPlan implements NormalizationPlan {
   public void execute(Admin admin) {
     LOG.info("Executing splitting normalization plan: " + this);
     try {
-      admin.splitRegion(regionInfo.getRegionName());
-    } catch (IOException ex) {
+      admin.splitRegionAsync(regionInfo.getRegionName(), null).get();
+    } catch (Exception ex) {
       LOG.error("Error during region split: ", ex);
     }
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
index 120f11e..c1b8489 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
@@ -111,11 +111,7 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
       cleanExpiredMobFiles(tableName, family);
       return 0;
     } finally {
-      try {
-        admin.close();
-      } catch (IOException e) {
-        LOG.error("Failed to close the HBaseAdmin.", e);
-      }
+      admin.close();
       try {
         connection.close();
       } catch (IOException e) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
index fbf73f3..ae3cc15 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
@@ -663,8 +663,8 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati
     @Override
     public void run() {
       Preconditions.checkNotNull(server);
-      if (server.isStopped()
-          || (region.getTableDescriptor() != null && !region.getTableDescriptor().isCompactionEnabled())) {
+      if (server.isStopped() || (region.getTableDescriptor() != null &&
+        !region.getTableDescriptor().isCompactionEnabled())) {
         region.decrementCompactionsQueuedCount();
         return;
       }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java
index c90860d..8bafaf5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java
@@ -53,8 +53,8 @@ public class DelimitedKeyPrefixRegionSplitPolicy extends IncreasingToUpperBoundR
     // read the prefix length from the table descriptor
     String delimiterString = region.getTableDescriptor().getValue(DELIMITER_KEY);
     if (delimiterString == null || delimiterString.length() == 0) {
-      LOG.error(DELIMITER_KEY + " not specified for table " + region.getTableDescriptor().getTableName() +
-        ". Using default RegionSplitPolicy");
+      LOG.error(DELIMITER_KEY + " not specified for table " +
+        region.getTableDescriptor().getTableName() + ". Using default RegionSplitPolicy");
       return;
     }
     delimiter = Bytes.toBytes(delimiterString);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
index adb04c7..d8f7af6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
@@ -130,7 +130,7 @@ public final class SnapshotManifest {
   /**
    * Return a SnapshotManifest instance with the information already loaded in-memory.
    *    SnapshotManifest manifest = SnapshotManifest.open(...)
-   *    TableDescriptor htd = manifest.getTableDescriptor()
+   *    TableDescriptor htd = manifest.getDescriptor()
    *    for (SnapshotRegionManifest regionManifest: manifest.getRegionManifests())
    *      hri = regionManifest.getRegionInfo()
    *      for (regionManifest.getFamilyFiles())
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index 40f4aa6..cf98d4a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -90,7 +90,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
@@ -99,6 +98,7 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
@@ -1202,13 +1202,13 @@ public final class Canary implements Tool {
 
       if (this.useRegExp) {
         Pattern pattern = null;
-        TableDescriptor[] tds = null;
+        List<TableDescriptor> tds = null;
         Set<String> tmpTables = new TreeSet<>();
         try {
           LOG.debug(String.format("reading list of tables"));
-          tds = this.admin.listTables(pattern);
+          tds = this.admin.listTableDescriptors(pattern);
           if (tds == null) {
-            tds = new TableDescriptor[0];
+            tds = Collections.emptyList();
           }
           for (String monitorTarget : monitorTargets) {
             pattern = Pattern.compile(monitorTarget);
@@ -1293,7 +1293,7 @@ public final class Canary implements Tool {
       }
       int numberOfCoveredServers = serverSet.size();
       if (numberOfCoveredServers < numberOfServers) {
-        admin.balancer();
+        admin.balance();
       }
     }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 9e5f9e8..1a00945 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -2291,7 +2291,7 @@ public class HBaseFsck extends Configured implements Closeable {
     if (hi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
       return;
     }
-    int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication();
+    int numReplicas = admin.getDescriptor(hi.getTableName()).getRegionReplication();
     for (int i = 1; i < numReplicas; i++) {
       if (hi.getPrimaryHRIForDeployedReplica() == null) continue;
       RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
@@ -2344,7 +2344,7 @@ public class HBaseFsck extends Configured implements Closeable {
     get.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER);
     // also get the locations of the replicas to close if the primary region is being closed
     if (hi.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
-      int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication();
+      int numReplicas = admin.getDescriptor(hi.getTableName()).getRegionReplication();
       for (int i = 0; i < numReplicas; i++) {
         get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(i));
         get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(i));
@@ -2395,7 +2395,7 @@ public class HBaseFsck extends Configured implements Closeable {
 
       // also assign replicas if needed (do it only when this call operates on a primary replica)
       if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) return;
-      int replicationCount = admin.getTableDescriptor(hri.getTable()).getRegionReplication();
+      int replicationCount = admin.getDescriptor(hri.getTable()).getRegionReplication();
       for (int i = 1; i < replicationCount; i++) {
         hri = RegionReplicaUtil.getRegionInfoForReplica(hri, i);
         HbckInfo h = regionInfoMap.get(hri.getEncodedName());
@@ -2512,7 +2512,7 @@ public class HBaseFsck extends Configured implements Closeable {
           }
         }
         LOG.info("Patching hbase:meta with .regioninfo: " + hbi.getHdfsHRI());
-        int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
+        int numReplicas = admin.getDescriptor(hbi.getTableName()).getRegionReplication();
         HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
             admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
               .getLiveServerMetrics().keySet(), numReplicas);
@@ -2540,7 +2540,7 @@ public class HBaseFsck extends Configured implements Closeable {
         }
 
         LOG.info("Patching hbase:meta with with .regioninfo: " + hbi.getHdfsHRI());
-        int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
+        int numReplicas = admin.getDescriptor(hbi.getTableName()).getRegionReplication();
         HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
             admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
               .getLiveServerMetrics().keySet(), numReplicas);
@@ -3660,7 +3660,7 @@ public class HBaseFsck extends Configured implements Closeable {
         metaRegions.put(value.getReplicaId(), value);
       }
     }
-    int metaReplication = admin.getTableDescriptor(TableName.META_TABLE_NAME)
+    int metaReplication = admin.getDescriptor(TableName.META_TABLE_NAME)
         .getRegionReplication();
     boolean noProblem = true;
     // There will be always entries in regionInfoMap corresponding to hbase:meta & its replicas
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp b/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp
index d105e10..0b9165b 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp
@@ -400,12 +400,12 @@
     </div>
 
     <% if (rsGroupTables != null && rsGroupTables.size() > 0) {
-        HTableDescriptor[] tables = null;
+        List<TableDescriptor> tables;
         try (Admin admin = master.getConnection().getAdmin()) {
-            tables = master.isInitialized() ? admin.listTables((Pattern)null, true) : null;
+            tables = master.isInitialized() ? admin.listTableDescriptors((Pattern)null, true) : null;
         }
          Map<TableName, HTableDescriptor> tableDescriptors
-            = Stream.of(tables).collect(Collectors.toMap(TableDescriptor::getTableName, p -> p));
+            = tables.stream().collect(Collectors.toMap(TableDescriptor::getTableName, p -> new HTableDescriptor(p)));
     %>
          <table class="table table-striped">
          <tr>
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index d3eaa77..ced5b44 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -172,7 +172,7 @@ if ( fqtn != null ) {
     %> Compact request accepted. <%
     } else if (action.equals("merge")) {
         if (left != null && left.length() > 0 && right != null && right.length() > 0) {
-            admin.mergeRegions(Bytes.toBytesBinary(left), Bytes.toBytesBinary(right), false);
+            admin.mergeRegionsAsync(Bytes.toBytesBinary(left), Bytes.toBytesBinary(right), false);
         }
         %> Merge request accepted. <%
     }
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 74d168c..516c3d8 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
@@ -1821,31 +1821,14 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   }
 
   /**
-   * Modify a table, synchronous. Waiting logic similar to that of {@code admin.rb#alter_status}.
+   * Modify a table, synchronous.
+   * @deprecated just use {@link Admin#modifyTable(TableDescriptor)} directly as it is synchronous
+   *             now.
    */
-  @SuppressWarnings("serial")
+  @Deprecated
   public static void modifyTableSync(Admin admin, TableDescriptor desc)
       throws IOException, InterruptedException {
     admin.modifyTable(desc);
-    Pair<Integer, Integer> status = new Pair<Integer, Integer>() {{
-      setFirst(0);
-      setSecond(0);
-    }};
-    int i = 0;
-    do {
-      status = admin.getAlterStatus(desc.getTableName());
-      if (status.getSecond() != 0) {
-        LOG.debug(status.getSecond() - status.getFirst() + "/" + status.getSecond()
-          + " regions updated.");
-        Thread.sleep(1 * 1000L);
-      } else {
-        LOG.debug("All regions updated.");
-        break;
-      }
-    } while (status.getFirst() != 0 && i++ < 500);
-    if (status.getFirst() != 0) {
-      throw new IOException("Failed to update all regions even after 500 seconds.");
-    }
   }
 
   /**
@@ -1854,9 +1837,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   public static void setReplicas(Admin admin, TableName table, int replicaCount)
       throws IOException, InterruptedException {
     admin.disableTable(table);
-    HTableDescriptor desc = new HTableDescriptor(admin.getTableDescriptor(table));
+    HTableDescriptor desc = new HTableDescriptor(admin.getDescriptor(table));
     desc.setRegionReplication(replicaCount);
-    admin.modifyTable(desc.getTableName(), desc);
+    admin.modifyTable(desc);
     admin.enableTable(table);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
index 53fd849..635ba8b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
@@ -22,7 +22,6 @@ import java.util.EnumSet;
 import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.Waiter.Predicate;
@@ -83,25 +82,6 @@ public class TestClientClusterStatus {
   }
 
   @Test
-  public void testDefaults() throws Exception {
-    ClusterStatus origin = ADMIN.getClusterStatus();
-    ClusterStatus defaults
-        = new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.allOf(Option.class)));
-    checkPbObjectNotNull(origin);
-    checkPbObjectNotNull(defaults);
-    Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
-    Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
-    Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
-    Assert.assertTrue(origin.getBackupMastersSize() == defaults.getBackupMastersSize());
-    Assert.assertTrue(origin.getDeadServersSize() == defaults.getDeadServersSize());
-    Assert.assertTrue(origin.getRegionsCount() == defaults.getRegionsCount());
-    Assert.assertTrue(origin.getServersSize() == defaults.getServersSize());
-    Assert.assertTrue(origin.getMasterInfoPort() == defaults.getMasterInfoPort());
-    Assert.assertTrue(origin.equals(defaults));
-    Assert.assertTrue(origin.getServersName().size() == defaults.getServersName().size());
-  }
-
-  @Test
   public void testNone() throws Exception {
     ClusterMetrics status0 = ADMIN.getClusterMetrics(EnumSet.allOf(Option.class));
     ClusterMetrics status1 = ADMIN.getClusterMetrics(EnumSet.noneOf(Option.class));
@@ -206,8 +186,8 @@ public class TestClientClusterStatus {
   public void testObserver() throws IOException {
     int preCount = MyObserver.PRE_COUNT.get();
     int postCount = MyObserver.POST_COUNT.get();
-    Assert.assertTrue(Stream.of(ADMIN.getClusterStatus().getMasterCoprocessors())
-        .anyMatch(s -> s.equals(MyObserver.class.getSimpleName())));
+    Assert.assertTrue(ADMIN.getClusterMetrics().getMasterCoprocessorNames().stream()
+      .anyMatch(s -> s.equals(MyObserver.class.getSimpleName())));
     Assert.assertEquals(preCount + 1, MyObserver.PRE_COUNT.get());
     Assert.assertEquals(postCount + 1, MyObserver.POST_COUNT.get());
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
index 5d5bc50..f062e58 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
@@ -23,14 +23,17 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.Callable;
+import java.util.regex.Pattern;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -84,7 +87,7 @@ public class TestNamespace {
 
   @Before
   public void beforeMethod() throws IOException {
-    for (HTableDescriptor desc : admin.listTables(prefix+".*")) {
+    for (TableDescriptor desc : admin.listTableDescriptors(Pattern.compile(prefix + ".*"))) {
       admin.disableTable(desc.getTableName());
       admin.deleteTable(desc.getTableName());
     }
@@ -112,14 +115,14 @@ public class TestNamespace {
     //verify existence of system tables
     Set<TableName> systemTables = Sets.newHashSet(
         TableName.META_TABLE_NAME);
-    HTableDescriptor[] descs =
-        admin.listTableDescriptorsByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE.getName());
-    assertEquals(systemTables.size(), descs.length);
-    for (HTableDescriptor desc : descs) {
+    List<TableDescriptor> descs = admin.listTableDescriptorsByNamespace(
+      Bytes.toBytes(NamespaceDescriptor.SYSTEM_NAMESPACE.getName()));
+    assertEquals(systemTables.size(), descs.size());
+    for (TableDescriptor desc : descs) {
       assertTrue(systemTables.contains(desc.getTableName()));
     }
     //verify system tables aren't listed
-    assertEquals(0, admin.listTables().length);
+    assertEquals(0, admin.listTableDescriptors().size());
 
     //Try creating default and system namespaces.
     boolean exceptionCaught = false;
@@ -189,15 +192,15 @@ public class TestNamespace {
     admin.createNamespace(NamespaceDescriptor.create(nsName).build());
     TEST_UTIL.createTable(tableName, Bytes.toBytes(nsName));
     TEST_UTIL.createTable(tableNameFoo,Bytes.toBytes(nsName));
-    assertEquals(2, admin.listTables().length);
+    assertEquals(2, admin.listTableDescriptors().size());
     assertNotNull(admin
-        .getTableDescriptor(tableName));
+        .getDescriptor(tableName));
     assertNotNull(admin
-        .getTableDescriptor(tableNameFoo));
+        .getDescriptor(tableNameFoo));
     //remove namespace and verify
     admin.disableTable(tableName);
     admin.deleteTable(tableName);
-    assertEquals(1, admin.listTables().length);
+    assertEquals(1, admin.listTableDescriptors().size());
   }
 
   @Test
@@ -222,7 +225,7 @@ public class TestNamespace {
         new Path(master.getMasterFileSystem().getRootDir(),
             new Path(HConstants.BASE_NAMESPACE_DIR,
                 new Path(nsName, desc.getTableName().getQualifierAsString())))));
-    assertEquals(1, admin.listTables().length);
+    assertEquals(1, admin.listTableDescriptors().size());
 
     //verify non-empty namespace can't be removed
     try {
@@ -253,7 +256,7 @@ public class TestNamespace {
     HColumnDescriptor colDesc = new HColumnDescriptor("cf1");
     desc.addFamily(colDesc);
     admin.createTable(desc);
-    assertTrue(admin.listTables().length == 1);
+    assertTrue(admin.listTableDescriptors().size() == 1);
     admin.disableTable(desc.getTableName());
     admin.deleteTable(desc.getTableName());
   }
@@ -265,7 +268,7 @@ public class TestNamespace {
     HColumnDescriptor colDesc = new HColumnDescriptor("cf1");
     desc.addFamily(colDesc);
     admin.createTable(desc);
-    assertEquals(0, admin.listTables().length);
+    assertEquals(0, admin.listTableDescriptors().size());
     assertTrue(admin.tableExists(tableName));
     admin.disableTable(desc.getTableName());
     admin.deleteTable(desc.getTableName());
@@ -338,15 +341,16 @@ public class TestNamespace {
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(prefix + "ns1", name.getMethodName()));
     htd.addFamily(new HColumnDescriptor("family1"));
     admin.createTable(htd);
-    HTableDescriptor[] htds = admin.listTableDescriptorsByNamespace(prefix + "ns1");
+    List<TableDescriptor> htds =
+      admin.listTableDescriptorsByNamespace(Bytes.toBytes(prefix + "ns1"));
     assertNotNull("Should have not returned null", htds);
-    assertEquals("Should have returned non-empty array", 1, htds.length);
+    assertEquals("Should have returned non-empty array", 1, htds.size());
 
     // get table descriptors for non-existing namespace
     runWithExpectedException(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        admin.listTableDescriptorsByNamespace("non_existant_namespace");
+        admin.listTableDescriptorsByNamespace(Bytes.toBytes("non_existant_namespace"));
         return null;
       }
     }, NamespaceNotFoundException.class);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
index cc62cbb..bf47e5b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
@@ -118,7 +118,7 @@ public class TestPartialResultsFromClientSide {
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().setLong(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, timeout);
     TEST_UTIL.startMiniCluster(MINICLUSTER_SIZE);
-    TEST_UTIL.getAdmin().setBalancerRunning(false, true);
+    TEST_UTIL.getAdmin().balancerSwitch(false, true);
     TABLE = createTestTable(TABLE_NAME, ROWS, FAMILIES, QUALIFIERS, VALUE);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
index a390aca..0241a4c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
@@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -70,7 +71,7 @@ public class TestRegionLoad {
     UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", MSG_INTERVAL);
     UTIL.startMiniCluster(4);
     admin = UTIL.getAdmin();
-    admin.setBalancerRunning(false, true);
+    admin.balancerSwitch(false, true);
     createTables();
   }
 
@@ -95,7 +96,7 @@ public class TestRegionLoad {
     // Check if regions match with the regionLoad from the server
     for (ServerName serverName : admin
         .getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()) {
-      List<HRegionInfo> regions = admin.getOnlineRegions(serverName);
+      List<RegionInfo> regions = admin.getRegions(serverName);
       LOG.info("serverName=" + serverName + ", regions=" +
           regions.stream().map(r -> r.getRegionNameAsString()).collect(Collectors.toList()));
       Collection<RegionLoad> regionLoads = admin.getRegionMetrics(serverName)
@@ -108,7 +109,7 @@ public class TestRegionLoad {
 
     // Check if regionLoad matches the table's regions and nothing is missed
     for (TableName table : new TableName[]{TABLE_1, TABLE_2, TABLE_3}) {
-      List<HRegionInfo> tableRegions = admin.getTableRegions(table);
+      List<RegionInfo> tableRegions = admin.getRegions(table);
 
       List<RegionLoad> regionLoads = Lists.newArrayList();
       for (ServerName serverName : admin
@@ -159,23 +160,21 @@ public class TestRegionLoad {
     assertEquals("regionLoads from SN should be empty", 0, regionLoads.size());
   }
 
-  private void checkRegionsAndRegionLoads(Collection<HRegionInfo> regions,
+  private void checkRegionsAndRegionLoads(Collection<RegionInfo> regions,
       Collection<RegionLoad> regionLoads) {
-
     for (RegionLoad load : regionLoads) {
       assertNotNull(load.regionLoadPB);
     }
 
-    assertEquals("No of regions and regionloads doesn't match",
-        regions.size(), regionLoads.size());
+    assertEquals("No of regions and regionloads doesn't match", regions.size(), regionLoads.size());
 
     Map<byte[], RegionLoad> regionLoadMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
     for (RegionLoad regionLoad : regionLoads) {
       regionLoadMap.put(regionLoad.getName(), regionLoad);
     }
-    for (HRegionInfo info : regions) {
+    for (RegionInfo info : regions) {
       assertTrue("Region not in regionLoadMap region:" + info.getRegionNameAsString() +
-          " regionMap: " + regionLoadMap, regionLoadMap.containsKey(info.getRegionName()));
+        " regionMap: " + regionLoadMap, regionLoadMap.containsKey(info.getRegionName()));
     }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientNormalTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientNormalTestBase.java
index 3e26076..cb3ecd5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientNormalTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientNormalTestBase.java
@@ -34,7 +34,7 @@ public class CloneSnapshotFromClientNormalTestBase extends CloneSnapshotFromClie
     testCloneSnapshot(clonedTableName, emptySnapshot, 0);
   }
 
-  private void testCloneSnapshot(TableName tableName, byte[] snapshotName, int snapshotRows)
+  private void testCloneSnapshot(TableName tableName, String snapshotName, int snapshotRows)
       throws IOException, InterruptedException {
     // create a new table from snapshot
     admin.cloneSnapshot(snapshotName, tableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientTestBase.java
index c74ec1d..00cc1a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientTestBase.java
@@ -40,10 +40,10 @@ public class CloneSnapshotFromClientTestBase {
 
   protected final byte[] FAMILY = Bytes.toBytes("cf");
 
-  protected byte[] emptySnapshot;
-  protected byte[] snapshotName0;
-  protected byte[] snapshotName1;
-  protected byte[] snapshotName2;
+  protected String emptySnapshot;
+  protected String snapshotName0;
+  protected String snapshotName1;
+  protected String snapshotName2;
   protected TableName tableName;
   protected int snapshot0Rows;
   protected int snapshot1Rows;
@@ -87,10 +87,10 @@ public class CloneSnapshotFromClientTestBase {
 
     long tid = System.currentTimeMillis();
     tableName = TableName.valueOf(getValidMethodName() + tid);
-    emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
-    snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
-    snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
-    snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
+    emptySnapshot = "emptySnaptb-" + tid;
+    snapshotName0 = "snaptb0-" + tid;
+    snapshotName1 = "snaptb1-" + tid;
+    snapshotName2 = "snaptb2-" + tid;
 
     createTableAndSnapshots();
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientSimpleTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientSimpleTestBase.java
index f4f2698..729679c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientSimpleTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientSimpleTestBase.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 
 public class RestoreSnapshotFromClientSimpleTestBase extends RestoreSnapshotFromClientTestBase {
@@ -63,7 +62,7 @@ public class RestoreSnapshotFromClientSimpleTestBase extends RestoreSnapshotFrom
 
   @Test
   public void testCorruptedSnapshot() throws IOException, InterruptedException {
-    SnapshotTestingUtils.corruptSnapshot(TEST_UTIL, Bytes.toString(snapshotName0));
+    SnapshotTestingUtils.corruptSnapshot(TEST_UTIL, snapshotName0);
     TableName cloneName =
       TableName.valueOf(getValidMethodName() + "-" + System.currentTimeMillis());
     try {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientTestBase.java
index 047640f..666d67d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientTestBase.java
@@ -42,10 +42,10 @@ public class RestoreSnapshotFromClientTestBase {
   protected final byte[] TEST_FAMILY2 = Bytes.toBytes("cf2");
 
   protected TableName tableName;
-  protected byte[] emptySnapshot;
-  protected byte[] snapshotName0;
-  protected byte[] snapshotName1;
-  protected byte[] snapshotName2;
+  protected String emptySnapshot;
+  protected String snapshotName0;
+  protected String snapshotName1;
+  protected String snapshotName2;
   protected int snapshot0Rows;
   protected int snapshot1Rows;
   protected Admin admin;
@@ -84,10 +84,10 @@ public class RestoreSnapshotFromClientTestBase {
 
     long tid = System.currentTimeMillis();
     tableName = TableName.valueOf(getValidMethodName() + "-" + tid);
-    emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
-    snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
-    snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
-    snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
+    emptySnapshot = "emptySnaptb-" + tid;
+    snapshotName0 = "snaptb0-" + tid;
+    snapshotName1 = "snaptb1-" + tid;
+    snapshotName2 = "snaptb2-" + tid;
 
     // create Table and disable it
     createTable();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index 376aa92..d86be09 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
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Pattern;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -59,7 +60,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -86,7 +86,7 @@ public class TestAdmin1 {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestAdmin1.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private Admin admin;
+  private static Admin ADMIN;
 
   @Rule
   public TestName name = new TestName();
@@ -98,21 +98,16 @@ public class TestAdmin1 {
     TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
     TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
     TEST_UTIL.startMiniCluster(3);
+    ADMIN = TEST_UTIL.getAdmin();
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
   }
-
-  @Before
-  public void setUp() throws Exception {
-    this.admin = TEST_UTIL.getAdmin();
-  }
-
   @After
   public void tearDown() throws Exception {
-    for (HTableDescriptor htd : this.admin.listTables()) {
+    for (TableDescriptor htd : ADMIN.listTableDescriptors()) {
       TEST_UTIL.deleteTable(htd.getTableName());
     }
   }
@@ -122,7 +117,7 @@ public class TestAdmin1 {
     final TableName unknowntable = TableName.valueOf(name.getMethodName());
     Exception exception = null;
     try {
-      this.admin.compact(unknowntable);
+      ADMIN.compact(unknowntable);
     } catch (IOException e) {
       exception = e;
     }
@@ -130,7 +125,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.flush(unknowntable);
+      ADMIN.flush(unknowntable);
     } catch (IOException e) {
       exception = e;
     }
@@ -138,7 +133,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.split(unknowntable);
+      ADMIN.split(unknowntable);
     } catch (IOException e) {
       exception = e;
     }
@@ -153,7 +148,7 @@ public class TestAdmin1 {
     HColumnDescriptor nonexistentHcd = new HColumnDescriptor(nonexistentColumn);
     Exception exception = null;
     try {
-      this.admin.addColumnFamily(nonexistentTable, nonexistentHcd);
+      ADMIN.addColumnFamily(nonexistentTable, nonexistentHcd);
     } catch (IOException e) {
       exception = e;
     }
@@ -161,7 +156,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.deleteTable(nonexistentTable);
+      ADMIN.deleteTable(nonexistentTable);
     } catch (IOException e) {
       exception = e;
     }
@@ -169,7 +164,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.deleteColumnFamily(nonexistentTable, nonexistentColumn);
+      ADMIN.deleteColumnFamily(nonexistentTable, nonexistentColumn);
     } catch (IOException e) {
       exception = e;
     }
@@ -177,7 +172,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.disableTable(nonexistentTable);
+      ADMIN.disableTable(nonexistentTable);
     } catch (IOException e) {
       exception = e;
     }
@@ -185,7 +180,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.enableTable(nonexistentTable);
+      ADMIN.enableTable(nonexistentTable);
     } catch (IOException e) {
       exception = e;
     }
@@ -193,7 +188,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.modifyColumnFamily(nonexistentTable, nonexistentHcd);
+      ADMIN.modifyColumnFamily(nonexistentTable, nonexistentHcd);
     } catch (IOException e) {
       exception = e;
     }
@@ -203,7 +198,7 @@ public class TestAdmin1 {
     try {
       HTableDescriptor htd = new HTableDescriptor(nonexistentTable);
       htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-      this.admin.modifyTable(htd.getTableName(), htd);
+      ADMIN.modifyTable(htd);
     } catch (IOException e) {
       exception = e;
     }
@@ -214,11 +209,11 @@ public class TestAdmin1 {
     final TableName tableName = TableName.valueOf(name.getMethodName() + System.currentTimeMillis());
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(new HColumnDescriptor("cf"));
-    this.admin.createTable(htd);
+    ADMIN.createTable(htd);
     try {
       exception = null;
       try {
-        this.admin.deleteColumnFamily(htd.getTableName(), nonexistentHcd.getName());
+        ADMIN.deleteColumnFamily(htd.getTableName(), nonexistentHcd.getName());
       } catch (IOException e) {
         exception = e;
       }
@@ -227,15 +222,15 @@ public class TestAdmin1 {
 
       exception = null;
       try {
-        this.admin.modifyColumnFamily(htd.getTableName(), nonexistentHcd);
+        ADMIN.modifyColumnFamily(htd.getTableName(), nonexistentHcd);
       } catch (IOException e) {
         exception = e;
       }
       assertTrue("found=" + exception.getClass().getName(),
           exception instanceof InvalidFamilyOperationException);
     } finally {
-      this.admin.disableTable(tableName);
-      this.admin.deleteTable(tableName);
+      ADMIN.disableTable(tableName);
+      ADMIN.deleteTable(tableName);
     }
   }
 
@@ -253,7 +248,7 @@ public class TestAdmin1 {
     get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
     ht.get(get);
 
-    this.admin.disableTable(ht.getName());
+    ADMIN.disableTable(ht.getName());
     assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster()
         .getMaster().getTableStateManager().isTableState(
             ht.getName(), TableState.State.DISABLED));
@@ -281,7 +276,7 @@ public class TestAdmin1 {
       ok = true;
     }
     assertTrue(ok);
-    this.admin.enableTable(table);
+    ADMIN.enableTable(table);
     assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
         .getMaster().getTableStateManager().isTableState(
             ht.getName(), TableState.State.ENABLED));
@@ -322,7 +317,10 @@ public class TestAdmin1 {
     ht1.get(get);
     ht2.get(get);
 
-    this.admin.disableTables("testDisableAndEnableTable.*");
+    TableName[] tableNames = ADMIN.listTableNames(Pattern.compile("testDisableAndEnableTable.*"));
+    for (TableName tableName: tableNames) {
+      ADMIN.disableTable(tableName);
+    }
 
     // Test that tables are disabled
     get = new Get(row);
@@ -340,7 +338,9 @@ public class TestAdmin1 {
 
 
     assertTrue(ok);
-    this.admin.enableTables("testDisableAndEnableTable.*");
+    for (TableName tableName: tableNames) {
+      ADMIN.enableTable(tableName);
+    }
 
     // Test that tables are enabled
     try {
@@ -364,12 +364,12 @@ public class TestAdmin1 {
 
   @Test
   public void testCreateTable() throws IOException {
-    HTableDescriptor [] tables = admin.listTables();
-    int numTables = tables.length;
+    List<TableDescriptor> tables = ADMIN.listTableDescriptors();
+    int numTables = tables.size();
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
-    tables = this.admin.listTables();
-    assertEquals(numTables + 1, tables.length);
+    tables = ADMIN.listTableDescriptors();
+    assertEquals(numTables + 1, tables.size());
     assertTrue("Table must be enabled.",
         TEST_UTIL.getHBaseCluster().getMaster().getTableStateManager()
             .isTableState(tableName, TableState.State.ENABLED));
@@ -403,8 +403,8 @@ public class TestAdmin1 {
     assertEquals(3, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
 
     // Truncate & Verify
-    this.admin.disableTable(tableName);
-    this.admin.truncateTable(tableName, preserveSplits);
+    ADMIN.disableTable(tableName);
+    ADMIN.truncateTable(tableName, preserveSplits);
     table = TEST_UTIL.getConnection().getTable(tableName);
     try {
       assertEquals(0, TEST_UTIL.countRows(table));
@@ -427,7 +427,7 @@ public class TestAdmin1 {
     htd.addFamily(fam1);
     htd.addFamily(fam2);
     htd.addFamily(fam3);
-    this.admin.createTable(htd);
+    ADMIN.createTable(htd);
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
     TableDescriptor confirmedHtd = table.getDescriptor();
     assertEquals(0, TableDescriptor.COMPARATOR.compare(htd, confirmedHtd));
@@ -441,19 +441,19 @@ public class TestAdmin1 {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(fam1);
-    this.admin.createTable(htd);
+    ADMIN.createTable(htd);
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
-    long ts = this.admin.getLastMajorCompactionTimestamp(tableName);
+    long ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
     assertEquals(0, ts);
     Put p = new Put(Bytes.toBytes("row1"));
     p.addColumn(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
     table.put(p);
-    ts = this.admin.getLastMajorCompactionTimestamp(tableName);
+    ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
     // no files written -> no data
     assertEquals(0, ts);
 
-    this.admin.flush(tableName);
-    ts = this.admin.getLastMajorCompactionTimestamp(tableName);
+    ADMIN.flush(tableName);
+    ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
     // still 0, we flushed a file, but no major compaction happened
     assertEquals(0, ts);
 
@@ -461,30 +461,30 @@ public class TestAdmin1 {
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
       regionName = l.getAllRegionLocations().get(0).getRegionInfo().getRegionName();
     }
-    long ts1 = this.admin.getLastMajorCompactionTimestampForRegion(regionName);
+    long ts1 = ADMIN.getLastMajorCompactionTimestampForRegion(regionName);
     assertEquals(ts, ts1);
     p = new Put(Bytes.toBytes("row2"));
     p.addColumn(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
     table.put(p);
-    this.admin.flush(tableName);
-    ts = this.admin.getLastMajorCompactionTimestamp(tableName);
+    ADMIN.flush(tableName);
+    ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
     // make sure the region API returns the same value, as the old file is still around
     assertEquals(ts1, ts);
 
     TEST_UTIL.compact(tableName, true);
     table.put(p);
     // forces a wait for the compaction
-    this.admin.flush(tableName);
-    ts = this.admin.getLastMajorCompactionTimestamp(tableName);
+    ADMIN.flush(tableName);
+    ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
     // after a compaction our earliest timestamp will have progressed forward
     assertTrue(ts > ts1);
 
     // region api still the same
-    ts1 = this.admin.getLastMajorCompactionTimestampForRegion(regionName);
+    ts1 = ADMIN.getLastMajorCompactionTimestampForRegion(regionName);
     assertEquals(ts, ts1);
     table.put(p);
-    this.admin.flush(tableName);
-    ts = this.admin.getLastMajorCompactionTimestamp(tableName);
+    ADMIN.flush(tableName);
+    ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
     assertEquals(ts, ts1);
     table.close();
   }
@@ -509,18 +509,18 @@ public class TestAdmin1 {
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
 
     // Make table read only
-    TableDescriptor htd = TableDescriptorBuilder.newBuilder(this.admin.getDescriptor(tableName))
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
       .setReadOnly(true).build();
-    admin.modifyTable(htd);
+    ADMIN.modifyTable(htd);
 
     // try to modify the read only table now
-    htd = TableDescriptorBuilder.newBuilder(this.admin.getDescriptor(tableName))
+    htd = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
       .setCompactionEnabled(false).build();
-    admin.modifyTable(htd);
+    ADMIN.modifyTable(htd);
     // Delete the table
-    this.admin.disableTable(tableName);
-    this.admin.deleteTable(tableName);
-    assertFalse(this.admin.tableExists(tableName));
+    ADMIN.disableTable(tableName);
+    ADMIN.deleteTable(tableName);
+    assertFalse(ADMIN.tableExists(tableName));
   }
 
   @Test(expected = TableNotDisabledException.class)
@@ -529,17 +529,17 @@ public class TestAdmin1 {
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
 
     // Modify region replication count
-    TableDescriptor htd = TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName))
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
         .setRegionReplication(3).build();
     try {
       // try to modify the region replication count without disabling the table
-      admin.modifyTable(htd);
+      ADMIN.modifyTable(htd);
       fail("Expected an exception");
     } finally {
       // Delete the table
-      admin.disableTable(tableName);
-      admin.deleteTable(tableName);
-      assertFalse(admin.tableExists(tableName));
+      ADMIN.disableTable(tableName);
+      ADMIN.deleteTable(tableName);
+      assertFalse(ADMIN.tableExists(tableName));
     }
   }
 
@@ -549,14 +549,14 @@ public class TestAdmin1 {
   @Test
   public void testOnlineChangeTableSchema() throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    HTableDescriptor [] tables = admin.listTables();
-    int numTables = tables.length;
+    List<TableDescriptor> tables = ADMIN.listTableDescriptors();
+    int numTables = tables.size();
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
-    tables = this.admin.listTables();
-    assertEquals(numTables + 1, tables.length);
+    tables = ADMIN.listTableDescriptors();
+    assertEquals(numTables + 1, tables.size());
 
     // FIRST, do htabledescriptor changes.
-    HTableDescriptor htd = this.admin.getTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
     // Make a copy and assert copy is good.
     HTableDescriptor copy = new HTableDescriptor(htd);
     assertTrue(htd.equals(copy));
@@ -571,12 +571,12 @@ public class TestAdmin1 {
     copy.setValue(key, key);
     boolean expectedException = false;
     try {
-      admin.modifyTable(tableName, copy);
+      ADMIN.modifyTable(copy);
     } catch (TableNotDisabledException re) {
       expectedException = true;
     }
     assertFalse(expectedException);
-    HTableDescriptor modifiedHtd = new HTableDescriptor(this.admin.getTableDescriptor(tableName));
+    HTableDescriptor modifiedHtd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
     assertFalse(htd.equals(modifiedHtd));
     assertTrue(copy.equals(modifiedHtd));
     assertEquals(newFlushSize, modifiedHtd.getMemStoreFlushSize());
@@ -592,44 +592,44 @@ public class TestAdmin1 {
     final byte [] hcdName = hcd.getName();
     expectedException = false;
     try {
-      this.admin.modifyColumnFamily(tableName, hcd);
+      ADMIN.modifyColumnFamily(tableName, hcd);
     } catch (TableNotDisabledException re) {
       expectedException = true;
     }
     assertFalse(expectedException);
-    modifiedHtd = this.admin.getTableDescriptor(tableName);
+    modifiedHtd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
     HColumnDescriptor modifiedHcd = modifiedHtd.getFamily(hcdName);
     assertEquals(newMaxVersions, modifiedHcd.getMaxVersions());
 
     // Try adding a column
-    assertFalse(this.admin.isTableDisabled(tableName));
+    assertFalse(ADMIN.isTableDisabled(tableName));
     final String xtracolName = "xtracol";
     HColumnDescriptor xtracol = new HColumnDescriptor(xtracolName);
     xtracol.setValue(xtracolName, xtracolName);
     expectedException = false;
     try {
-      this.admin.addColumnFamily(tableName, xtracol);
+      ADMIN.addColumnFamily(tableName, xtracol);
     } catch (TableNotDisabledException re) {
       expectedException = true;
     }
     // Add column should work even if the table is enabled
     assertFalse(expectedException);
-    modifiedHtd = this.admin.getTableDescriptor(tableName);
+    modifiedHtd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
     hcd = modifiedHtd.getFamily(xtracol.getName());
     assertTrue(hcd != null);
     assertTrue(hcd.getValue(xtracolName).equals(xtracolName));
 
     // Delete the just-added column.
-    this.admin.deleteColumnFamily(tableName, xtracol.getName());
-    modifiedHtd = this.admin.getTableDescriptor(tableName);
+    ADMIN.deleteColumnFamily(tableName, xtracol.getName());
+    modifiedHtd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
     hcd = modifiedHtd.getFamily(xtracol.getName());
     assertTrue(hcd == null);
 
     // Delete the table
-    this.admin.disableTable(tableName);
-    this.admin.deleteTable(tableName);
-    this.admin.listTables();
-    assertFalse(this.admin.tableExists(tableName));
+    ADMIN.disableTable(tableName);
+    ADMIN.deleteTable(tableName);
+    ADMIN.listTableDescriptors();
+    assertFalse(ADMIN.tableExists(tableName));
   }
 
   protected void verifyRoundRobinDistribution(ClusterConnection c, RegionLocator regionLocator, int
@@ -667,7 +667,7 @@ public class TestAdmin1 {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc);
+    ADMIN.createTable(desc);
     List<HRegionLocation> regions;
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
       regions = l.getAllRegionLocations();
@@ -677,7 +677,7 @@ public class TestAdmin1 {
     TableName TABLE_2 = TableName.valueOf(tableName.getNameAsString() + "_2");
     desc = new HTableDescriptor(TABLE_2);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, new byte[][]{new byte[]{42}});
+    ADMIN.createTable(desc, new byte[][]{new byte[]{42}});
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_2)) {
       regions = l.getAllRegionLocations();
       assertEquals("Table should have only 2 region", 2, regions.size());
@@ -686,7 +686,7 @@ public class TestAdmin1 {
     TableName TABLE_3 = TableName.valueOf(tableName.getNameAsString() + "_3");
     desc = new HTableDescriptor(TABLE_3);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 3);
+    ADMIN.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 3);
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_3)) {
       regions = l.getAllRegionLocations();
       assertEquals("Table should have only 3 region", 3, regions.size());
@@ -696,7 +696,7 @@ public class TestAdmin1 {
     desc = new HTableDescriptor(TABLE_4);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     try {
-      admin.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 2);
+      ADMIN.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 2);
       fail("Should not be able to create a table with only 2 regions using this API.");
     } catch (IllegalArgumentException eae) {
     // Expected
@@ -705,7 +705,7 @@ public class TestAdmin1 {
     TableName TABLE_5 = TableName.valueOf(tableName.getNameAsString() + "_5");
     desc = new HTableDescriptor(TABLE_5);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, new byte[] { 1 }, new byte[] { 127 }, 16);
+    ADMIN.createTable(desc, new byte[] { 1 }, new byte[] { 127 }, 16);
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_5)) {
       regions = l.getAllRegionLocations();
       assertEquals("Table should have 16 region", 16, regions.size());
@@ -715,7 +715,6 @@ public class TestAdmin1 {
   @Test
   public void testCreateTableWithRegions() throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-
     byte [][] splitKeys = {
         new byte [] { 1, 1, 1 },
         new byte [] { 2, 2, 2 },
@@ -731,9 +730,9 @@ public class TestAdmin1 {
 
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, splitKeys);
+    ADMIN.createTable(desc, splitKeys);
 
-    boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys);
+    boolean tableAvailable = ADMIN.isTableAvailable(tableName);
     assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable);
 
     List<HRegionLocation> regions;
@@ -799,8 +798,7 @@ public class TestAdmin1 {
 
     desc = new HTableDescriptor(TABLE_2);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin = TEST_UTIL.getAdmin();
-    admin.createTable(desc, startKey, endKey, expectedRegions);
+    ADMIN.createTable(desc, startKey, endKey, expectedRegions);
 
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_2)) {
       regions = l.getAllRegionLocations();
@@ -854,8 +852,7 @@ public class TestAdmin1 {
 
     desc = new HTableDescriptor(TABLE_3);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin = TEST_UTIL.getAdmin();
-    admin.createTable(desc, startKey, endKey, expectedRegions);
+    ADMIN.createTable(desc, startKey, endKey, expectedRegions);
 
 
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_3)) {
@@ -880,7 +877,7 @@ public class TestAdmin1 {
     desc = new HTableDescriptor(TABLE_4);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     try {
-      admin.createTable(desc, splitKeys);
+      ADMIN.createTable(desc, splitKeys);
       assertTrue("Should not be able to create this table because of " +
           "duplicate split keys", false);
     } catch(IllegalArgumentException iae) {
@@ -889,21 +886,6 @@ public class TestAdmin1 {
   }
 
   @Test
-  public void testTableAvailableWithRandomSplitKeys() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(new HColumnDescriptor("col"));
-    byte[][] splitKeys = new byte[1][];
-    splitKeys = new byte [][] {
-        new byte [] { 1, 1, 1 },
-        new byte [] { 2, 2, 2 }
-    };
-    admin.createTable(desc);
-    boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys);
-    assertFalse("Table should be created with 1 row in META", tableAvailable);
-  }
-
-  @Test
   public void testCreateTableWithOnlyEmptyStartRow() throws IOException {
     final byte[] tableName = Bytes.toBytes(name.getMethodName());
     byte[][] splitKeys = new byte[1][];
@@ -911,7 +893,7 @@ public class TestAdmin1 {
     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor("col"));
     try {
-      admin.createTable(desc, splitKeys);
+      ADMIN.createTable(desc, splitKeys);
       fail("Test case should fail as empty split key is passed.");
     } catch (IllegalArgumentException e) {
     }
@@ -927,7 +909,7 @@ public class TestAdmin1 {
     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor("col"));
     try {
-      admin.createTable(desc, splitKeys);
+      ADMIN.createTable(desc, splitKeys);
       fail("Test case should fail as empty split key is passed.");
     } catch (IllegalArgumentException e) {
       LOG.info("Expected ", e);
@@ -938,10 +920,10 @@ public class TestAdmin1 {
   public void testTableExist() throws IOException {
     final TableName table = TableName.valueOf(name.getMethodName());
     boolean exist;
-    exist = this.admin.tableExists(table);
+    exist = ADMIN.tableExists(table);
     assertEquals(false, exist);
     TEST_UTIL.createTable(table, HConstants.CATALOG_FAMILY);
-    exist = this.admin.tableExists(table);
+    exist = ADMIN.tableExists(table);
     assertEquals(true, exist);
   }
 
@@ -979,7 +961,7 @@ public class TestAdmin1 {
     int expectedRegions = splitKeys.length + 1;
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, splitKeys);
+    ADMIN.createTable(desc, splitKeys);
 
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
       List<HRegionLocation> regions = l.getAllRegionLocations();
@@ -988,9 +970,9 @@ public class TestAdmin1 {
           "Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
           expectedRegions, regions.size());
       // Disable table.
-      admin.disableTable(tableName);
+      ADMIN.disableTable(tableName);
       // Enable table, use retain assignment to assign regions.
-      admin.enableTable(tableName);
+      ADMIN.enableTable(tableName);
       List<HRegionLocation> regions2 = l.getAllRegionLocations();
 
       // Check the assignment.
@@ -1041,7 +1023,7 @@ public class TestAdmin1 {
     for (int i = 0; i < rowCounts.length; i++) {
       sb.append("_").append(Integer.toString(rowCounts[i]));
     }
-    assertFalse(admin.tableExists(tableName));
+    assertFalse(ADMIN.tableExists(tableName));
     try (final Table table = TEST_UTIL.createTable(tableName, familyNames,
       numVersions, blockSize);
       final RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
@@ -1089,7 +1071,7 @@ public class TestAdmin1 {
 
       // Split the table
       if (async) {
-        this.admin.split(tableName, splitPoint);
+        ADMIN.split(tableName, splitPoint);
         final AtomicInteger count = new AtomicInteger(0);
         Thread t = new Thread("CheckForSplit") {
           @Override public void run() {
@@ -1122,7 +1104,7 @@ public class TestAdmin1 {
         t.join();
       } else {
         // Sync split region, no need to create a thread to check
-        ((HBaseAdmin)admin).splitRegionSync(m.get(0).getRegionInfo().getRegionName(), splitPoint);
+        ADMIN.splitRegionAsync(m.get(0).getRegion().getRegionName(), splitPoint).get();
       }
 
       // Verify row count
@@ -1214,7 +1196,7 @@ 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().splitRegion(regions.get(1).getFirst().getRegionName());
+      TEST_UTIL.getAdmin().splitRegionAsync(regions.get(1).getFirst().getRegionName(), null).get();
     } catch (IllegalArgumentException ex) {
       gotException = true;
     }
@@ -1287,54 +1269,54 @@ public class TestAdmin1 {
   public void testEnableDisableAddColumnDeleteColumn() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
-    while (!this.admin.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
+    while (!ADMIN.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
       Thread.sleep(10);
     }
-    this.admin.disableTable(tableName);
+    ADMIN.disableTable(tableName);
     try {
       TEST_UTIL.getConnection().getTable(tableName);
     } catch (org.apache.hadoop.hbase.DoNotRetryIOException e) {
       //expected
     }
 
-    this.admin.addColumnFamily(tableName, new HColumnDescriptor("col2"));
-    this.admin.enableTable(tableName);
+    ADMIN.addColumnFamily(tableName, new HColumnDescriptor("col2"));
+    ADMIN.enableTable(tableName);
     try {
-      this.admin.deleteColumnFamily(tableName, Bytes.toBytes("col2"));
+      ADMIN.deleteColumnFamily(tableName, Bytes.toBytes("col2"));
     } catch (TableNotDisabledException e) {
       LOG.info(e.toString(), e);
     }
-    this.admin.disableTable(tableName);
-    this.admin.deleteTable(tableName);
+    ADMIN.disableTable(tableName);
+    ADMIN.deleteTable(tableName);
   }
 
   @Test
   public void testDeleteLastColumnFamily() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
-    while (!this.admin.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
+    while (!ADMIN.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
       Thread.sleep(10);
     }
 
     // test for enabled table
     try {
-      this.admin.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
+      ADMIN.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
       fail("Should have failed to delete the only column family of a table");
     } catch (InvalidFamilyOperationException ex) {
       // expected
     }
 
     // test for disabled table
-    this.admin.disableTable(tableName);
+    ADMIN.disableTable(tableName);
 
     try {
-      this.admin.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
+      ADMIN.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
       fail("Should have failed to delete the only column family of a table");
     } catch (InvalidFamilyOperationException ex) {
       // expected
     }
 
-    this.admin.deleteTable(tableName);
+    ADMIN.deleteTable(tableName);
   }
 
   /*
@@ -1365,7 +1347,7 @@ public class TestAdmin1 {
     puts.add(p);
     try {
       table.put(puts);
-      admin.flush(tableName);
+      ADMIN.flush(tableName);
 
       List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(tableName);
       for (HRegion r : regions) {
@@ -1385,9 +1367,9 @@ public class TestAdmin1 {
         }
       }
     } finally {
-      if (admin.isTableEnabled(tableName)) {
-        this.admin.disableTable(tableName);
-        this.admin.deleteTable(tableName);
+      if (ADMIN.isTableEnabled(tableName)) {
+        ADMIN.disableTable(tableName);
+        ADMIN.deleteTable(tableName);
       }
     }
   }
@@ -1410,29 +1392,29 @@ public class TestAdmin1 {
       RegionInfo regionB;
 
       // merge with full name
-      tableRegions = admin.getRegions(tableName);
-      assertEquals(3, admin.getTableRegions(tableName).size());
+      tableRegions = ADMIN.getRegions(tableName);
+      assertEquals(3, ADMIN.getRegions(tableName).size());
       regionA = tableRegions.get(0);
       regionB = tableRegions.get(1);
       // TODO convert this to version that is synchronous (See HBASE-16668)
-      admin.mergeRegionsAsync(regionA.getRegionName(), regionB.getRegionName(), false)
+      ADMIN.mergeRegionsAsync(regionA.getRegionName(), regionB.getRegionName(), false)
           .get(60, TimeUnit.SECONDS);
 
-      assertEquals(2, admin.getTableRegions(tableName).size());
+      assertEquals(2, ADMIN.getRegions(tableName).size());
 
       // merge with encoded name
-      tableRegions = admin.getRegions(tableName);
+      tableRegions = ADMIN.getRegions(tableName);
       regionA = tableRegions.get(0);
       regionB = tableRegions.get(1);
       // TODO convert this to version that is synchronous (See HBASE-16668)
-      admin.mergeRegionsAsync(
+      ADMIN.mergeRegionsAsync(
         regionA.getEncodedNameAsBytes(), regionB.getEncodedNameAsBytes(), false)
           .get(60, TimeUnit.SECONDS);
 
-      assertEquals(1, admin.getTableRegions(tableName).size());
+      assertEquals(1, ADMIN.getRegions(tableName).size());
     } finally {
-      this.admin.disableTable(tableName);
-      this.admin.deleteTable(tableName);
+      ADMIN.disableTable(tableName);
+      ADMIN.deleteTable(tableName);
     }
   }
 
@@ -1451,16 +1433,16 @@ public class TestAdmin1 {
       p.addColumn(Bytes.toBytes("f"), q1, v1);
       table.put(p);
     }
-    this.admin.flush(tableName);
+    ADMIN.flush(tableName);
     try {
-      this.admin.split(tableName, Bytes.toBytes("row5"));
+      ADMIN.split(tableName, Bytes.toBytes("row5"));
       Threads.sleep(10000);
     } catch (Exception e) {
       // Nothing to do.
     }
     // Split should not happen.
     List<RegionInfo> allRegions = MetaTableAccessor.getTableRegions(
-        this.admin.getConnection(), tableName, true);
+        ADMIN.getConnection(), tableName, true);
     assertEquals(1, allRegions.size());
   }
 
@@ -1502,15 +1484,15 @@ public class TestAdmin1 {
             .setTimeToLive(TTL)
             .build()
         ).build();
-    admin.createTable(tableDesc, splitKeys);
+    ADMIN.createTable(tableDesc, splitKeys);
 
     assertEquals(NUM_REGIONS, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
     assertTrue("Table should be created with splitKyes + 1 rows in META",
-        admin.isTableAvailable(tableName, splitKeys));
+      ADMIN.isTableAvailable(tableName));
 
     // clone & Verify
-    admin.cloneTableSchema(tableName, newTableName, preserveSplits);
-    TableDescriptor newTableDesc = admin.getDescriptor(newTableName);
+    ADMIN.cloneTableSchema(tableName, newTableName, preserveSplits);
+    TableDescriptor newTableDesc = ADMIN.getDescriptor(newTableName);
 
     assertEquals(NUM_FAMILYS, newTableDesc.getColumnFamilyCount());
     assertEquals(BLOCK_SIZE, newTableDesc.getColumnFamily(FAMILY_1).getBlocksize());
@@ -1521,7 +1503,7 @@ public class TestAdmin1 {
     if (preserveSplits) {
       assertEquals(NUM_REGIONS, TEST_UTIL.getHBaseCluster().getRegions(newTableName).size());
       assertTrue("New table should be created with splitKyes + 1 rows in META",
-          admin.isTableAvailable(newTableName, splitKeys));
+          ADMIN.isTableAvailable(newTableName));
     } else {
       assertEquals(1, TEST_UTIL.getHBaseCluster().getRegions(newTableName).size());
     }
@@ -1533,7 +1515,7 @@ public class TestAdmin1 {
     final TableName newTableName = TableName.valueOf(tableName.getNameAsString() + "_new");
     // test for non-existent source table
     try {
-      admin.cloneTableSchema(tableName, newTableName, false);
+      ADMIN.cloneTableSchema(tableName, newTableName, false);
       fail("Should have failed to create a new table by cloning non-existent source table.");
     } catch (TableNotFoundException ex) {
       // expected
@@ -1549,7 +1531,7 @@ public class TestAdmin1 {
     TEST_UTIL.createTable(newTableName, FAMILY_0);
     // test for existent destination table
     try {
-      admin.cloneTableSchema(tableName, newTableName, false);
+      ADMIN.cloneTableSchema(tableName, newTableName, false);
       fail("Should have failed to create a existent table.");
     } catch (TableExistsException ex) {
       // expected
@@ -1564,15 +1546,15 @@ public class TestAdmin1 {
         .setRegionReplication(5)
         .build();
 
-    admin.createTable(desc);
+    ADMIN.createTable(desc);
 
     int maxFileSize = 10000000;
     TableDescriptor newDesc = TableDescriptorBuilder.newBuilder(desc)
         .setMaxFileSize(maxFileSize)
         .build();
 
-    admin.modifyTable(newDesc);
-    TableDescriptor newTableDesc = admin.getDescriptor(tableName);
+    ADMIN.modifyTable(newDesc);
+    TableDescriptor newTableDesc = ADMIN.getDescriptor(tableName);
     assertEquals(maxFileSize, newTableDesc.getMaxFileSize());
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index 2c14eaf..b928750 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
@@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -89,7 +88,7 @@ public class TestAdmin2 {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestAdmin2.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private Admin admin;
+  private static Admin ADMIN;
 
   @Rule
   public TestName name = new TestName();
@@ -103,6 +102,7 @@ public class TestAdmin2 {
     TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 30);
     TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
     TEST_UTIL.startMiniCluster(3);
+    ADMIN = TEST_UTIL.getAdmin();
   }
 
   @AfterClass
@@ -110,14 +110,9 @@ public class TestAdmin2 {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Before
-  public void setUp() throws Exception {
-    this.admin = TEST_UTIL.getHBaseAdmin();
-  }
-
   @After
   public void tearDown() throws Exception {
-    for (HTableDescriptor htd : this.admin.listTables()) {
+    for (TableDescriptor htd : ADMIN.listTableDescriptors()) {
       TEST_UTIL.deleteTable(htd.getTableName());
     }
   }
@@ -126,7 +121,7 @@ public class TestAdmin2 {
   public void testCreateBadTables() throws IOException {
     String msg = null;
     try {
-      this.admin.createTable(new HTableDescriptor(TableName.META_TABLE_NAME));
+      ADMIN.createTable(new HTableDescriptor(TableName.META_TABLE_NAME));
     } catch(TableExistsException e) {
       msg = e.toString();
     }
@@ -141,7 +136,7 @@ public class TestAdmin2 {
     Thread [] threads = new Thread [count];
     final AtomicInteger successes = new AtomicInteger(0);
     final AtomicInteger failures = new AtomicInteger(0);
-    final Admin localAdmin = this.admin;
+    final Admin localAdmin = ADMIN;
     for (int i = 0; i < count; i++) {
       threads[i] = new Thread(Integer.toString(i)) {
         @Override
@@ -186,8 +181,8 @@ public class TestAdmin2 {
     HTableDescriptor htd2 = new HTableDescriptor(TableName.valueOf(name));
     htd1.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     htd2.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(htd1);
-    admin.createTable(htd2);
+    ADMIN.createTable(htd1);
+    ADMIN.createTable(htd2);
     // Before fix, below would fail throwing a NoServerForRegionException.
     TEST_UTIL.getConnection().getTable(htd2.getTableName()).close();
   }
@@ -197,7 +192,6 @@ public class TestAdmin2 {
    * Thus creating of table with lots of regions can cause RPC timeout
    * After the fix to make createTable truly async, RPC timeout shouldn't be an
    * issue anymore
-   * @throws Exception
    */
   @Test
   public void testCreateTableRPCTimeOut() throws Exception {
@@ -238,7 +232,6 @@ public class TestAdmin2 {
   /**
    * Test that user table names can contain '-' and '.' so long as they do not
    * start with same. HBASE-771
-   * @throws IOException
    */
   @Test
   public void testTableNames() throws IOException {
@@ -267,7 +260,6 @@ public class TestAdmin2 {
 
   /**
    * For HADOOP-2579
-   * @throws IOException
    */
   @Test (expected=TableExistsException.class)
   public void testTableExistsExceptionWithATable() throws IOException {
@@ -278,41 +270,35 @@ public class TestAdmin2 {
 
   /**
    * Can't disable a table if the table isn't in enabled state
-   * @throws IOException
    */
   @Test (expected=TableNotEnabledException.class)
   public void testTableNotEnabledExceptionWithATable() throws IOException {
     final TableName name = TableName.valueOf(this.name.getMethodName());
     TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY).close();
-    this.admin.disableTable(name);
-    this.admin.disableTable(name);
+    ADMIN.disableTable(name);
+    ADMIN.disableTable(name);
   }
 
   /**
    * Can't enable a table if the table isn't in disabled state
-   * @throws IOException
    */
   @Test (expected=TableNotDisabledException.class)
   public void testTableNotDisabledExceptionWithATable() throws IOException {
     final TableName name = TableName.valueOf(this.name.getMethodName());
-    Table t = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
-    try {
-    this.admin.enableTable(name);
-    }finally {
-       t.close();
+    try (Table t = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY)) {
+      ADMIN.enableTable(name);
     }
   }
 
   /**
    * For HADOOP-2579
-   * @throws IOException
    */
-  @Test (expected=TableNotFoundException.class)
+  @Test(expected = TableNotFoundException.class)
   public void testTableNotFoundExceptionWithoutAnyTables() throws IOException {
-    TableName tableName = TableName
-        .valueOf("testTableNotFoundExceptionWithoutAnyTables");
-    Table ht = TEST_UTIL.getConnection().getTable(tableName);
-    ht.get(new Get(Bytes.toBytes("e")));
+    TableName tableName = TableName.valueOf("testTableNotFoundExceptionWithoutAnyTables");
+    try (Table ht = TEST_UTIL.getConnection().getTable(tableName)) {
+      ht.get(new Get(Bytes.toBytes("e")));
+    }
   }
 
   @Test
@@ -326,7 +312,7 @@ public class TestAdmin2 {
     for (RegionInfo regionInfo : onlineRegions) {
       if (!regionInfo.getTable().isSystemTable()) {
         info = regionInfo;
-        admin.unassign(regionInfo.getRegionName(), true);
+        ADMIN.unassign(regionInfo.getRegionName(), true);
       }
     }
     boolean isInList = ProtobufUtil.getOnlineRegions(
@@ -356,7 +342,7 @@ public class TestAdmin2 {
         if (regionInfo.getRegionNameAsString().contains(name)) {
           info = regionInfo;
           try {
-            admin.unassign(Bytes.toBytes("sample"), true);
+            ADMIN.unassign(Bytes.toBytes("sample"), true);
           } catch (UnknownRegionException nsre) {
             // expected, ignore it
           }
@@ -380,7 +366,7 @@ public class TestAdmin2 {
       if (!regionInfo.isMetaRegion()) {
         if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion2")) {
           info = regionInfo;
-          admin.unassign(regionInfo.getRegionName(), true);
+          ADMIN.unassign(regionInfo.getRegionName(), true);
         }
       }
     }
@@ -418,7 +404,7 @@ public class TestAdmin2 {
     HColumnDescriptor hcd = new HColumnDescriptor("value");
     htd.addFamily(hcd);
 
-    admin.createTable(htd, null);
+    ADMIN.createTable(htd, null);
   }
 
   /**
@@ -438,9 +424,9 @@ public class TestAdmin2 {
 
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, startKey, endKey, expectedRegions);
+    ADMIN.createTable(desc, startKey, endKey, expectedRegions);
 
-    List<RegionInfo> RegionInfos = admin.getRegions(tableName);
+    List<RegionInfo> RegionInfos = ADMIN.getRegions(tableName);
 
     assertEquals("Tried to create " + expectedRegions + " regions " +
         "but only found " + RegionInfos.size(),
@@ -479,7 +465,7 @@ public class TestAdmin2 {
     for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
       r.flush(true);
     }
-    admin.rollWALWriter(regionServer.getServerName());
+    ADMIN.rollWALWriter(regionServer.getServerName());
     int count = AbstractFSWALProvider.getNumRolledLogFiles(regionServer.getWAL(null));
     LOG.info("after flushing all regions and rolling logs there are " +
         count + " log files");
@@ -536,7 +522,7 @@ public class TestAdmin2 {
     // Create the test table and open it
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc);
+    ADMIN.createTable(desc);
     Table table = TEST_UTIL.getConnection().getTable(tableName);
 
     HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
@@ -585,7 +571,7 @@ public class TestAdmin2 {
   @Test
   public void testDisableCatalogTable() throws Exception {
     try {
-      this.admin.disableTable(TableName.META_TABLE_NAME);
+      ADMIN.disableTable(TableName.META_TABLE_NAME);
       fail("Expected to throw ConstraintException");
     } catch (ConstraintException e) {
     }
@@ -601,13 +587,13 @@ public class TestAdmin2 {
   @Test
   public void testIsEnabledOrDisabledOnUnknownTable() throws Exception {
     try {
-      admin.isTableEnabled(TableName.valueOf(name.getMethodName()));
+      ADMIN.isTableEnabled(TableName.valueOf(name.getMethodName()));
       fail("Test should fail if isTableEnabled called on unknown table.");
     } catch (IOException e) {
     }
 
     try {
-      admin.isTableDisabled(TableName.valueOf(name.getMethodName()));
+      ADMIN.isTableDisabled(TableName.valueOf(name.getMethodName()));
       fail("Test should fail if isTableDisabled called on unknown table.");
     } catch (IOException e) {
     }
@@ -636,46 +622,46 @@ public class TestAdmin2 {
 
   @Test
   public void testBalancer() throws Exception {
-    boolean initialState = admin.isBalancerEnabled();
+    boolean initialState = ADMIN.isBalancerEnabled();
 
     // Start the balancer, wait for it.
-    boolean prevState = admin.setBalancerRunning(!initialState, true);
+    boolean prevState = ADMIN.balancerSwitch(!initialState, true);
 
     // The previous state should be the original state we observed
     assertEquals(initialState, prevState);
 
     // Current state should be opposite of the original
-    assertEquals(!initialState, admin.isBalancerEnabled());
+    assertEquals(!initialState, ADMIN.isBalancerEnabled());
 
     // Reset it back to what it was
-    prevState = admin.setBalancerRunning(initialState, true);
+    prevState = ADMIN.balancerSwitch(initialState, true);
 
     // The previous state should be the opposite of the initial state
     assertEquals(!initialState, prevState);
     // Current state should be the original state again
-    assertEquals(initialState, admin.isBalancerEnabled());
+    assertEquals(initialState, ADMIN.isBalancerEnabled());
   }
 
   @Test
   public void testRegionNormalizer() throws Exception {
-    boolean initialState = admin.isNormalizerEnabled();
+    boolean initialState = ADMIN.isNormalizerEnabled();
 
     // flip state
-    boolean prevState = admin.setNormalizerRunning(!initialState);
+    boolean prevState = ADMIN.normalizerSwitch(!initialState);
 
     // The previous state should be the original state we observed
     assertEquals(initialState, prevState);
 
     // Current state should be opposite of the original
-    assertEquals(!initialState, admin.isNormalizerEnabled());
+    assertEquals(!initialState, ADMIN.isNormalizerEnabled());
 
     // Reset it back to what it was
-    prevState = admin.setNormalizerRunning(initialState);
+    prevState = ADMIN.normalizerSwitch(initialState);
 
     // The previous state should be the opposite of the initial state
     assertEquals(!initialState, prevState);
     // Current state should be the original state again
-    assertEquals(initialState, admin.isNormalizerEnabled());
+    assertEquals(initialState, ADMIN.isNormalizerEnabled());
   }
 
   @Test
@@ -683,32 +669,32 @@ public class TestAdmin2 {
     Random randomGenerator = new Random();
     long procId = randomGenerator.nextLong();
 
-    boolean abortResult = admin.abortProcedure(procId, true);
+    boolean abortResult = ADMIN.abortProcedure(procId, true);
     assertFalse(abortResult);
   }
 
   @Test
   public void testGetProcedures() throws Exception {
-    String procList = admin.getProcedures();
+    String procList = ADMIN.getProcedures();
     assertTrue(procList.startsWith("["));
   }
 
   @Test
   public void testGetLocks() throws Exception {
-    String lockList = admin.getLocks();
+    String lockList = ADMIN.getLocks();
     assertTrue(lockList.startsWith("["));
   }
 
   @Test
   public void testDecommissionRegionServers() throws Exception {
-    List<ServerName> decommissionedRegionServers = admin.listDecommissionedRegionServers();
+    List<ServerName> decommissionedRegionServers = ADMIN.listDecommissionedRegionServers();
     assertTrue(decommissionedRegionServers.isEmpty());
 
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.createMultiRegionTable(tableName, Bytes.toBytes("f"), 6);
 
     ArrayList<ServerName> clusterRegionServers =
-        new ArrayList<>(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+        new ArrayList<>(ADMIN.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
           .getLiveServerMetrics().keySet());
 
     assertEquals(3, clusterRegionServers.size());
@@ -718,8 +704,8 @@ public class TestAdmin2 {
     // leaving one online.
     int i;
     for (i = 0; i < clusterRegionServers.size(); i++) {
-      List<RegionInfo> regionsOnServer = admin.getRegions(clusterRegionServers.get(i));
-      if (admin.getRegions(clusterRegionServers.get(i)).stream().anyMatch(p -> p.isMetaRegion())) {
+      List<RegionInfo> regionsOnServer = ADMIN.getRegions(clusterRegionServers.get(i));
+      if (ADMIN.getRegions(clusterRegionServers.get(i)).stream().anyMatch(p -> p.isMetaRegion())) {
         serversToDecommssion.put(clusterRegionServers.get(i), regionsOnServer);
         break;
       }
@@ -728,13 +714,13 @@ public class TestAdmin2 {
     clusterRegionServers.remove(i);
     // Get another server to decommission.
     serversToDecommssion.put(clusterRegionServers.get(0),
-      admin.getRegions(clusterRegionServers.get(0)));
+      ADMIN.getRegions(clusterRegionServers.get(0)));
 
     ServerName remainingServer = clusterRegionServers.get(1);
 
     // Decommission
-    admin.decommissionRegionServers(new ArrayList<ServerName>(serversToDecommssion.keySet()), true);
-    assertEquals(2, admin.listDecommissionedRegionServers().size());
+    ADMIN.decommissionRegionServers(new ArrayList<ServerName>(serversToDecommssion.keySet()), true);
+    assertEquals(2, ADMIN.listDecommissionedRegionServers().size());
 
     // Verify the regions have been off the decommissioned servers, all on the one
     // remaining server.
@@ -748,9 +734,9 @@ public class TestAdmin2 {
     for (ServerName server : serversToDecommssion.keySet()) {
       List<byte[]> encodedRegionNames = serversToDecommssion.get(server).stream()
           .map(region -> region.getEncodedNameAsBytes()).collect(Collectors.toList());
-      admin.recommissionRegionServer(server, encodedRegionNames);
+      ADMIN.recommissionRegionServer(server, encodedRegionNames);
     }
-    assertTrue(admin.listDecommissionedRegionServers().isEmpty());
+    assertTrue(ADMIN.listDecommissionedRegionServers().isEmpty());
     // Verify the regions have been moved to the recommissioned servers
     for (ServerName server : serversToDecommssion.keySet()) {
       for (RegionInfo region : serversToDecommssion.get(server)) {
@@ -770,7 +756,7 @@ public class TestAdmin2 {
       table.put(new Put(Bytes.toBytes(i)).addColumn(Bytes.toBytes("f"), Bytes.toBytes("q"),
         Bytes.toBytes(i)));
     }
-    admin.flush(tableName);
+    ADMIN.flush(tableName);
 
     HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
     List<HRegion> regions = rs.getRegions(tableName);
@@ -783,7 +769,7 @@ public class TestAdmin2 {
     Assert.assertNotNull(store);
     Assert.assertEquals(expectedStoreFilesSize, store.getSize());
 
-    ClusterConnection conn = ((ClusterConnection) admin.getConnection());
+    ClusterConnection conn = ((ClusterConnection) ADMIN.getConnection());
     HBaseRpcController controller = conn.getRpcControllerFactory().newController();
     for (int i = 0; i < 10; i++) {
       RegionInfo ri =
@@ -801,26 +787,26 @@ public class TestAdmin2 {
     TEST_UTIL.createTable(tableName, Bytes.toBytes("f"));
 
     // get the original table region count
-    List<RegionInfo> regions = admin.getRegions(tableName);
+    List<RegionInfo> regions = ADMIN.getRegions(tableName);
     int originalCount = regions.size();
     assertEquals(1, originalCount);
 
     // split the table and wait until region count increases
-    admin.split(tableName, Bytes.toBytes(3));
+    ADMIN.split(tableName, Bytes.toBytes(3));
     TEST_UTIL.waitFor(30000, new Predicate<Exception>() {
 
       @Override
       public boolean evaluate() throws Exception {
-        return admin.getRegions(tableName).size() > originalCount;
+        return ADMIN.getRegions(tableName).size() > originalCount;
       }
     });
 
     // do some table modification
-    TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName))
+    TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
         .setMaxFileSize(11111111)
         .build();
-    admin.modifyTable(tableDesc);
-    assertEquals(11111111, admin.getDescriptor(tableName).getMaxFileSize());
+    ADMIN.modifyTable(tableDesc);
+    assertEquals(11111111, ADMIN.getDescriptor(tableName).getMaxFileSize());
   }
 
   @Test
@@ -830,7 +816,7 @@ public class TestAdmin2 {
       new byte[][] { Bytes.toBytes(3) });
 
     // assert we have at least 2 regions in the table
-    List<RegionInfo> regions = admin.getRegions(tableName);
+    List<RegionInfo> regions = ADMIN.getRegions(tableName);
     int originalCount = regions.size();
     assertTrue(originalCount >= 2);
 
@@ -838,20 +824,20 @@ public class TestAdmin2 {
     byte[] nameOfRegionB = regions.get(1).getEncodedNameAsBytes();
 
     // merge the table regions and wait until region count decreases
-    admin.mergeRegionsAsync(nameOfRegionA, nameOfRegionB, true);
+    ADMIN.mergeRegionsAsync(nameOfRegionA, nameOfRegionB, true);
     TEST_UTIL.waitFor(30000, new Predicate<Exception>() {
 
       @Override
       public boolean evaluate() throws Exception {
-        return admin.getRegions(tableName).size() < originalCount;
+        return ADMIN.getRegions(tableName).size() < originalCount;
       }
     });
 
     // do some table modification
-    TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName))
+    TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
         .setMaxFileSize(11111111)
         .build();
-    admin.modifyTable(tableDesc);
-    assertEquals(11111111, admin.getDescriptor(tableName).getMaxFileSize());
+    ADMIN.modifyTable(tableDesc);
+    assertEquals(11111111, ADMIN.getDescriptor(tableName).getMaxFileSize());
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAlwaysSetScannerId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAlwaysSetScannerId.java
index 2ede1dd..6c9c257 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAlwaysSetScannerId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAlwaysSetScannerId.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -61,7 +60,7 @@ public class TestAlwaysSetScannerId {
 
   private static final int COUNT = 10;
 
-  private static HRegionInfo HRI;
+  private static RegionInfo HRI;
 
   private static ClientProtos.ClientService.BlockingInterface STUB;
 
@@ -73,7 +72,7 @@ public class TestAlwaysSetScannerId {
         table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
       }
     }
-    HRI = UTIL.getAdmin().getTableRegions(TABLE_NAME).get(0);
+    HRI = UTIL.getAdmin().getRegions(TABLE_NAME).get(0);
     STUB = ((ConnectionImplementation) UTIL.getConnection())
         .getClient(UTIL.getHBaseCluster().getRegionServer(0).getServerName());
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
index 0488b38..28a53cd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
@@ -106,7 +106,7 @@ public class TestClientTimeouts {
           admin = connection.getAdmin();
           // run some admin commands
           HBaseAdmin.available(conf);
-          admin.setBalancerRunning(false, false);
+          admin.balancerSwitch(false, false);
         } catch (MasterNotRunningException ex) {
           // Since we are randomly throwing SocketTimeoutExceptions, it is possible to get
           // a MasterNotRunningException.  It's a bug if we get other exceptions.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
index 9523a61..9eec04a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
@@ -284,7 +284,7 @@ public class TestConnectionImplementation {
     TableName tableName = TableName.valueOf("HCM-testConnectionClose" + allowsInterrupt);
     TEST_UTIL.createTable(tableName, FAM_NAM).close();
 
-    boolean previousBalance = TEST_UTIL.getAdmin().setBalancerRunning(false, true);
+    boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true);
 
     Configuration c2 = new Configuration(TEST_UTIL.getConfiguration());
     // We want to work on a separate connection.
@@ -366,7 +366,7 @@ public class TestConnectionImplementation {
     table.close();
     connection.close();
     Assert.assertTrue("Unexpected exception is " + failed.get(), failed.get() == null);
-    TEST_UTIL.getAdmin().setBalancerRunning(previousBalance, true);
+    TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true);
   }
 
   /**
@@ -377,7 +377,7 @@ public class TestConnectionImplementation {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.createTable(tableName, FAM_NAM).close();
     int idleTime =  20000;
-    boolean previousBalance = TEST_UTIL.getAdmin().setBalancerRunning(false, true);
+    boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true);
 
     Configuration c2 = new Configuration(TEST_UTIL.getConfiguration());
     // We want to work on a separate connection.
@@ -425,7 +425,7 @@ public class TestConnectionImplementation {
 
     connection.close();
     EnvironmentEdgeManager.reset();
-    TEST_UTIL.getAdmin().setBalancerRunning(previousBalance, true);
+    TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true);
   }
 
     /**
@@ -438,7 +438,7 @@ public class TestConnectionImplementation {
     final TableName tableName = TableName.valueOf(name.getMethodName());
 
     TEST_UTIL.createTable(tableName, FAM_NAM).close();
-    boolean previousBalance = TEST_UTIL.getAdmin().setBalancerRunning(false, true);
+    boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true);
 
     Configuration c2 = new Configuration(TEST_UTIL.getConfiguration());
     // We want to work on a separate connection.
@@ -493,7 +493,7 @@ public class TestConnectionImplementation {
     } finally {
       syncBlockingFilter.set(true);
       t.join();
-      TEST_UTIL.getAdmin().setBalancerRunning(previousBalance, true);
+      TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true);
     }
 
     table.close();
@@ -572,7 +572,7 @@ public class TestConnectionImplementation {
     assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
     assertNotNull(conn.getCachedLocation(TableName.valueOf(TABLE_NAME.getName()), ROW.clone()));
 
-    TEST_UTIL.getAdmin().setBalancerRunning(false, false);
+    TEST_UTIL.getAdmin().balancerSwitch(false, false);
     HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
 
     // We can wait for all regions to be online, that makes log reading easier when debugging
@@ -874,7 +874,7 @@ public class TestConnectionImplementation {
       conn.clearRegionCache(TABLE_NAME3);
       Assert.assertEquals(0, conn.getNumberOfCachedRegionLocations(TABLE_NAME3));
 
-      TEST_UTIL.getAdmin().setBalancerRunning(false, false);
+      TEST_UTIL.getAdmin().balancerSwitch(false, false);
       HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
 
       // We can wait for all regions to be online, that makes log reading easier when debugging
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 0b8e2ad..5a285ef 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
@@ -29,7 +29,6 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -4211,15 +4210,14 @@ public class TestFromClientSide {
       TEST_UTIL.createTable(tables[i], FAMILY);
     }
     Admin admin = TEST_UTIL.getAdmin();
-    HTableDescriptor[] ts = admin.listTables();
-    HashSet<HTableDescriptor> result = new HashSet<HTableDescriptor>(ts.length);
-    Collections.addAll(result, ts);
+    List<TableDescriptor> ts = admin.listTableDescriptors();
+    HashSet<TableDescriptor> result = new HashSet<>(ts);
     int size = result.size();
     assertTrue(size >= tables.length);
     for (int i = 0; i < tables.length && i < size; i++) {
       boolean found = false;
-      for (int j = 0; j < ts.length; j++) {
-        if (ts[j].getTableName().equals(tables[i])) {
+      for (int j = 0; j < ts.size(); j++) {
+        if (ts.get(j).getTableName().equals(tables[i])) {
           found = true;
           break;
         }
@@ -4334,7 +4332,7 @@ public class TestFromClientSide {
     for (HColumnDescriptor c : desc.getFamilies())
       c.setValue(attrName, attrValue);
     // update metadata for all regions of this table
-    admin.modifyTable(tableAname, desc);
+    admin.modifyTable(desc);
     // enable the table
     admin.enableTable(tableAname);
 
@@ -6364,7 +6362,7 @@ public class TestFromClientSide {
     byte[][] KEYS = HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE;
     Admin admin = TEST_UTIL.getAdmin();
     admin.createTable(htd, KEYS);
-    List<HRegionInfo> regions = admin.getTableRegions(htd.getTableName());
+    List<RegionInfo> regions = admin.getRegions(htd.getTableName());
 
     HRegionLocator locator =
         (HRegionLocator) admin.getConnection().getRegionLocator(htd.getTableName());
@@ -6372,7 +6370,7 @@ public class TestFromClientSide {
       List<RegionLocations> regionLocations = new ArrayList<>();
 
       // mock region locations coming from meta with multiple replicas
-      for (HRegionInfo region : regions) {
+      for (RegionInfo region : regions) {
         HRegionLocation[] arr = new HRegionLocation[regionReplication];
         for (int i = 0; i < arr.length; i++) {
           arr[i] = new HRegionLocation(RegionReplicaUtil.getRegionInfoForReplica(region, i), null);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index 1315d4a..2d021bb 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
@@ -64,7 +64,6 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
 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.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -121,23 +120,17 @@ public class TestFromClientSide3 {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @Before
   public void setUp() throws Exception {
     // Nothing to do.
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @After
   public void tearDown() throws Exception {
-    for (HTableDescriptor htd: TEST_UTIL.getAdmin().listTables()) {
+    for (TableDescriptor htd : TEST_UTIL.getAdmin().listTableDescriptors()) {
       LOG.info("Tear down, remove table=" + htd.getTableName());
       TEST_UTIL.deleteTable(htd.getTableName());
-  }
+    }
   }
 
   private void randomCFPuts(Table table, byte[] row, byte[] family, int nPuts)
@@ -325,12 +318,7 @@ public class TestFromClientSide3 {
       LOG.info("hbase.hstore.compaction.min should now be 5");
       HTableDescriptor htd = new HTableDescriptor(hTable.getTableDescriptor());
       htd.setValue("hbase.hstore.compaction.min", String.valueOf(5));
-      admin.modifyTable(tableName, htd);
-      Pair<Integer, Integer> st;
-      while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
-        LOG.debug(st.getFirst() + " regions left to update");
-        Thread.sleep(40);
-      }
+      admin.modifyTable(htd);
       LOG.info("alter status finished");
 
       // Create 3 more store files.
@@ -352,11 +340,7 @@ public class TestFromClientSide3 {
       HColumnDescriptor hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
       hcd.setValue("hbase.hstore.compaction.min", String.valueOf(2));
       htd.modifyFamily(hcd);
-      admin.modifyTable(tableName, htd);
-      while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
-        LOG.debug(st.getFirst() + " regions left to update");
-        Thread.sleep(40);
-      }
+      admin.modifyTable(htd);
       LOG.info("alter status finished");
 
       // Issue a compaction request
@@ -387,11 +371,7 @@ public class TestFromClientSide3 {
       hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
       hcd.setValue("hbase.hstore.compaction.min", null);
       htd.modifyFamily(hcd);
-      admin.modifyTable(tableName, htd);
-      while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
-        LOG.debug(st.getFirst() + " regions left to update");
-        Thread.sleep(40);
-      }
+      admin.modifyTable(htd);
       LOG.info("alter status finished");
       assertNull(hTable.getTableDescriptor().getFamily(FAMILY).getValue(
           "hbase.hstore.compaction.min"));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
index a1026a9..59ccc47 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
@@ -129,7 +129,7 @@ public class TestHBaseAdminNoCluster {
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.listTables();
+        admin.listTableDescriptors();
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@@ -153,11 +153,11 @@ public class TestHBaseAdminNoCluster {
       }
     });
 
-    // Admin.getTableDescriptor()
+    // Admin.getDescriptor()
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.getTableDescriptor(TableName.valueOf(name.getMethodName()));
+        admin.getDescriptor(TableName.valueOf(name.getMethodName()));
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@@ -171,7 +171,7 @@ public class TestHBaseAdminNoCluster {
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.getTableDescriptorsByTableName(new ArrayList<>());
+        admin.listTableDescriptors(new ArrayList<>());
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@@ -209,11 +209,11 @@ public class TestHBaseAdminNoCluster {
       }
     });
 
-    // Admin.setBalancerRunning()
+    // Admin.balancerSwitch()
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.setBalancerRunning(true, true);
+        admin.balancerSwitch(true, true);
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@@ -223,11 +223,11 @@ public class TestHBaseAdminNoCluster {
       }
     });
 
-    // Admin.balancer()
+    // Admin.balance()
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.balancer();
+        admin.balance();
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@@ -241,7 +241,7 @@ public class TestHBaseAdminNoCluster {
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.enableCatalogJanitor(true);
+        admin.catalogJanitorSwitch(true);
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@@ -255,7 +255,7 @@ public class TestHBaseAdminNoCluster {
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.runCatalogScan();
+        admin.runCatalogJanitor();
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
index 3e499bd..dfe147d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.CompatibilityFactory;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.MetricsHBaseServerSource;
 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
@@ -84,20 +83,14 @@ public class TestLeaseRenewal {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @Before
   public void setUp() throws Exception {
     // Nothing to do.
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @After
   public void tearDown() throws Exception {
-    for (HTableDescriptor htd : TEST_UTIL.getAdmin().listTables()) {
+    for (TableDescriptor htd : TEST_UTIL.getAdmin().listTableDescriptors()) {
       LOG.info("Tear down, remove table=" + htd.getTableName());
       TEST_UTIL.deleteTable(htd.getTableName());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 6c633a2..1c2814f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -157,7 +157,7 @@ public class TestMetaWithReplicas {
 
   @Test
   public void testMetaHTDReplicaCount() throws Exception {
-    assertTrue(TEST_UTIL.getAdmin().getTableDescriptor(TableName.META_TABLE_NAME)
+    assertTrue(TEST_UTIL.getAdmin().getDescriptor(TableName.META_TABLE_NAME)
         .getRegionReplication() == 3);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java
index 1cae7a0..b3b4e06 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.Optional;
 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.coprocessor.ObserverContext;
@@ -112,14 +113,14 @@ public class TestMobCloneSnapshotFromClientCloneLinksAfterDelete
     delayFlush = true;
     SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 20, FAMILY);
     long tid = System.currentTimeMillis();
-    byte[] snapshotName3 = Bytes.toBytes("snaptb3-" + tid);
+    String snapshotName3 = "snaptb3-" + tid;
     TableName clonedTableName3 =
       TableName.valueOf(name.getMethodName() + System.currentTimeMillis());
     admin.snapshot(snapshotName3, tableName);
     delayFlush = false;
     int snapshot3Rows = -1;
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
-      snapshot3Rows = TEST_UTIL.countRows(table);
+      snapshot3Rows = HBaseTestingUtility.countRows(table);
     }
     admin.cloneSnapshot(snapshotName3, clonedTableName3);
     admin.deleteSnapshot(snapshotName3);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
index a460fee..bf54449 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
@@ -96,7 +96,7 @@ public class TestMultiRespectsLimits {
       TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
         @Override
         public boolean evaluate() throws Exception {
-          return admin.getTableRegions(tableName).size() > 1;
+          return admin.getRegions(tableName).size() > 1;
         }
       });
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index a42b26d..89430f6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -670,7 +670,7 @@ public class TestReplicaWithCluster {
   // within configured hbase.client.metaReplicaCallTimeout.scan from primary meta region.
   @Test
   public void testGetRegionLocationFromPrimaryMetaRegion() throws IOException, InterruptedException {
-    HTU.getAdmin().setBalancerRunning(false, true);
+    HTU.getAdmin().balancerSwitch(false, true);
 
     ((ConnectionImplementation) HTU.getAdmin().getConnection()).setUseMetaReplicas(true);
 
@@ -690,7 +690,7 @@ public class TestReplicaWithCluster {
     } finally {
       RegionServerHostingPrimayMetaRegionSlowOrStopCopro.slowDownPrimaryMetaScan = false;
       ((ConnectionImplementation) HTU.getAdmin().getConnection()).setUseMetaReplicas(false);
-      HTU.getAdmin().setBalancerRunning(true, true);
+      HTU.getAdmin().balancerSwitch(true, true);
       HTU.getAdmin().disableTable(hdt.getTableName());
       HTU.deleteTable(hdt.getTableName());
     }
@@ -703,7 +703,7 @@ public class TestReplicaWithCluster {
   // with the primary meta region.
   @Test
   public void testReplicaGetWithPrimaryAndMetaDown() throws IOException, InterruptedException {
-    HTU.getAdmin().setBalancerRunning(false, true);
+    HTU.getAdmin().balancerSwitch(false, true);
 
     ((ConnectionImplementation)HTU.getAdmin().getConnection()).setUseMetaReplicas(true);
 
@@ -789,7 +789,7 @@ public class TestReplicaWithCluster {
     } finally {
       ((ConnectionImplementation)HTU.getAdmin().getConnection()).setUseMetaReplicas(false);
       RegionServerHostingPrimayMetaRegionSlowOrStopCopro.throwException = false;
-      HTU.getAdmin().setBalancerRunning(true, true);
+      HTU.getAdmin().balancerSwitch(true, true);
       HTU.getAdmin().disableTable(hdt.getTableName());
       HTU.deleteTable(hdt.getTableName());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanWithoutFetchingData.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanWithoutFetchingData.java
index f5c2176..b3295ac 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanWithoutFetchingData.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanWithoutFetchingData.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
@@ -65,7 +64,7 @@ public class TestScanWithoutFetchingData {
 
   private static final int COUNT = 10;
 
-  private static HRegionInfo HRI;
+  private static RegionInfo HRI;
 
   private static ClientProtos.ClientService.BlockingInterface STUB;
 
@@ -77,7 +76,7 @@ public class TestScanWithoutFetchingData {
         table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
       }
     }
-    HRI = UTIL.getAdmin().getTableRegions(TABLE_NAME).get(0);
+    HRI = UTIL.getAdmin().getRegions(TABLE_NAME).get(0);
     STUB = ((ConnectionImplementation) UTIL.getConnection())
         .getClient(UTIL.getHBaseCluster().getRegionServer(0).getServerName());
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
index 0562c90..6c58473 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
@@ -26,7 +26,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.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@@ -80,7 +79,7 @@ public class TestSnapshotCloneIndependence {
   private TableName cloneTableName;
   private int countOriginalTable;
   String snapshotNameAsString;
-  byte[] snapshotName;
+  String snapshotName;
 
   /**
    * Setup the config for the cluster and start it
@@ -127,7 +126,7 @@ public class TestSnapshotCloneIndependence {
     originalTableName = TableName.valueOf("test" + testName.getMethodName());
     cloneTableName = TableName.valueOf("test-clone-" + originalTableName);
     snapshotNameAsString = "snapshot_" + originalTableName;
-    snapshotName = Bytes.toBytes(snapshotNameAsString);
+    snapshotName = snapshotNameAsString;
 
     originalTable = createTable(originalTableName, TEST_FAM);
     loadData(originalTable, TEST_FAM);
@@ -297,21 +296,21 @@ public class TestSnapshotCloneIndependence {
    */
   private void runTestRegionOperationsIndependent() throws Exception {
     // Verify that region information is the same pre-split
-    ((ClusterConnection) UTIL.getConnection()).clearRegionCache();
-    List<HRegionInfo> originalTableHRegions = admin.getTableRegions(originalTableName);
+    UTIL.getConnection().clearRegionLocationCache();
+    List<RegionInfo> originalTableHRegions = admin.getRegions(originalTableName);
 
     final int originalRegionCount = originalTableHRegions.size();
-    final int cloneTableRegionCount = admin.getTableRegions(cloneTableName).size();
+    final int cloneTableRegionCount = admin.getRegions(cloneTableName).size();
     Assert.assertEquals(
       "The number of regions in the cloned table is different than in the original table.",
       originalRegionCount, cloneTableRegionCount);
 
     // Split a region on the parent table
-    admin.splitRegion(originalTableHRegions.get(0).getRegionName());
+    admin.splitRegionAsync(originalTableHRegions.get(0).getRegionName(), null).get();
     waitOnSplit(UTIL.getConnection(), originalTable, originalRegionCount);
 
     // Verify that the cloned table region is not split
-    final int cloneTableRegionCount2 = admin.getTableRegions(cloneTableName).size();
+    final int cloneTableRegionCount2 = admin.getRegions(cloneTableName).size();
     Assert.assertEquals(
       "The number of regions in the cloned table changed though none of its regions were split.",
       cloneTableRegionCount, cloneTableRegionCount2);
@@ -335,8 +334,9 @@ public class TestSnapshotCloneIndependence {
     // get a description of the cloned table
     // get a list of its families
     // assert that the family is there
-    HTableDescriptor originalTableDescriptor = originalTable.getTableDescriptor();
-    HTableDescriptor clonedTableDescriptor = admin.getTableDescriptor(cloneTableName);
+    HTableDescriptor originalTableDescriptor = new HTableDescriptor(originalTable.getDescriptor());
+    HTableDescriptor clonedTableDescriptor =
+      new HTableDescriptor(admin.getDescriptor(cloneTableName));
 
     Assert.assertTrue("The original family was not found. There is something wrong. ",
       originalTableDescriptor.hasFamily(TEST_FAM));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
index b46404f..ad4b84c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
@@ -146,19 +146,10 @@ public class TestSnapshotFromClient {
 
   /**
    * Test snapshotting not allowed hbase:meta and -ROOT-
-   * @throws Exception
    */
-  @Test
+  @Test(expected = IllegalArgumentException.class)
   public void testMetaTablesSnapshot() throws Exception {
-    Admin admin = UTIL.getAdmin();
-    byte[] snapshotName = Bytes.toBytes("metaSnapshot");
-
-    try {
-      admin.snapshot(snapshotName, TableName.META_TABLE_NAME);
-      fail("taking a snapshot of hbase:meta should not be allowed");
-    } catch (IllegalArgumentException e) {
-      // expected
-    }
+    UTIL.getAdmin().snapshot("metaSnapshot", TableName.META_TABLE_NAME);
   }
 
   /**
@@ -177,16 +168,16 @@ public class TestSnapshotFromClient {
     UTIL.loadTable(table, TEST_FAM);
     table.close();
 
-    byte[] snapshot1 = Bytes.toBytes("TableSnapshot1");
+    String snapshot1 = "TableSnapshot1";
     admin.snapshot(snapshot1, TABLE_NAME);
     LOG.debug("Snapshot1 completed.");
 
-    byte[] snapshot2 = Bytes.toBytes("TableSnapshot2");
+    String snapshot2 = "TableSnapshot2";
     admin.snapshot(snapshot2, TABLE_NAME);
     LOG.debug("Snapshot2 completed.");
 
     String snapshot3 = "3rdTableSnapshot";
-    admin.snapshot(Bytes.toBytes(snapshot3), TABLE_NAME);
+    admin.snapshot(snapshot3, TABLE_NAME);
     LOG.debug(snapshot3 + " completed.");
 
     // delete the first two snapshots
@@ -260,13 +251,13 @@ public class TestSnapshotFromClient {
     // make sure the table doesn't exist
     boolean fail = false;
     do {
-    try {
-      admin.getTableDescriptor(TableName.valueOf(tableName));
-      fail = true;
-          LOG.error("Table:" + tableName + " already exists, checking a new name");
-      tableName = tableName+"!";
-    } catch (TableNotFoundException e) {
-      fail = false;
+      try {
+        admin.getDescriptor(TableName.valueOf(tableName));
+        fail = true;
+        LOG.error("Table:" + tableName + " already exists, checking a new name");
+        tableName = tableName + "!";
+      } catch (TableNotFoundException e) {
+        fail = false;
       }
     } while (fail);
 
@@ -297,7 +288,7 @@ public class TestSnapshotFromClient {
       FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 
     // take a snapshot of the disabled table
-    byte[] snapshot = Bytes.toBytes("testOfflineTableSnapshotWithEmptyRegions");
+    String snapshot = "testOfflineTableSnapshotWithEmptyRegions";
     admin.snapshot(snapshot, TABLE_NAME);
     LOG.debug("Snapshot completed.");
 
@@ -342,7 +333,7 @@ public class TestSnapshotFromClient {
       LOG.debug("Snapshot2 completed.");
 
       String table2Snapshot1 = "Table2Snapshot1";
-      admin.snapshot(Bytes.toBytes(table2Snapshot1), tableName);
+      admin.snapshot(table2Snapshot1, tableName);
       LOG.debug(table2Snapshot1 + " completed.");
 
       List<SnapshotDescription> listTableSnapshots =
@@ -384,7 +375,7 @@ public class TestSnapshotFromClient {
       LOG.debug("Snapshot2 completed.");
 
       String table2Snapshot1 = "Table2Snapshot1";
-      admin.snapshot(Bytes.toBytes(table2Snapshot1), TABLE_NAME);
+      admin.snapshot(table2Snapshot1, TABLE_NAME);
       LOG.debug(table2Snapshot1 + " completed.");
 
       List<SnapshotDescription> listTableSnapshots =
@@ -427,7 +418,7 @@ public class TestSnapshotFromClient {
       LOG.debug("Snapshot2 completed.");
 
       String table2Snapshot1 = "Table2Snapshot1";
-      admin.snapshot(Bytes.toBytes(table2Snapshot1), tableName);
+      admin.snapshot(table2Snapshot1, tableName);
       LOG.debug(table2Snapshot1 + " completed.");
 
       Pattern tableNamePattern = Pattern.compile("test.*");
@@ -459,7 +450,7 @@ public class TestSnapshotFromClient {
       LOG.debug("Snapshot2 completed.");
 
       String table2Snapshot1 = "Table2Snapshot1";
-      admin.snapshot(Bytes.toBytes(table2Snapshot1), TABLE_NAME);
+      admin.snapshot(table2Snapshot1, TABLE_NAME);
       LOG.debug(table2Snapshot1 + " completed.");
 
       admin.deleteTableSnapshots(tableNamePattern, Pattern.compile("Table1.*"));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java
index ef52057..1af70cb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java
@@ -178,7 +178,7 @@ public class TestSnapshotMetadata {
     admin.createTable(htd);
     Table original = UTIL.getConnection().getTable(originalTableName);
     originalTableName = TableName.valueOf(sourceTableNameAsString);
-    originalTableDescriptor = admin.getTableDescriptor(originalTableName);
+    originalTableDescriptor = new HTableDescriptor(admin.getDescriptor(originalTableName));
     originalTableDescription = originalTableDescriptor.toStringCustomizedValues();
 
     original.close();
@@ -195,7 +195,7 @@ public class TestSnapshotMetadata {
     final TableName clonedTableName = TableName.valueOf(clonedTableNameAsString);
     final String snapshotNameAsString = "snapshot" + originalTableName
         + System.currentTimeMillis();
-    final byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
+    final String snapshotName = snapshotNameAsString;
 
     // restore the snapshot into a cloned table and examine the output
     List<byte[]> familiesList = new ArrayList<>();
@@ -207,7 +207,7 @@ public class TestSnapshotMetadata {
 
     admin.cloneSnapshot(snapshotName, clonedTableName);
     Table clonedTable = UTIL.getConnection().getTable(clonedTableName);
-    HTableDescriptor cloneHtd = admin.getTableDescriptor(clonedTableName);
+    HTableDescriptor cloneHtd = new HTableDescriptor(admin.getDescriptor(clonedTableName));
     assertEquals(
       originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString),
       cloneHtd.toStringCustomizedValues());
@@ -287,7 +287,6 @@ public class TestSnapshotMetadata {
     // take a "disabled" snapshot
     final String snapshotNameAsString = "snapshot" + originalTableName
         + System.currentTimeMillis();
-    final byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
 
     SnapshotTestingUtils.createSnapshotAndValidate(admin, originalTableName,
       familiesWithDataList, emptyFamiliesList, snapshotNameAsString, rootDir, fs,
@@ -303,7 +302,7 @@ public class TestSnapshotMetadata {
       HColumnDescriptor hcd = new HColumnDescriptor(newFamilyName);
       admin.addColumnFamily(originalTableName, hcd);
       assertTrue("New column family was not added.",
-        admin.getTableDescriptor(originalTableName).toString().contains(newFamilyNameAsString));
+        admin.getDescriptor(originalTableName).toString().contains(newFamilyNameAsString));
     }
 
     // restore it
@@ -311,16 +310,14 @@ public class TestSnapshotMetadata {
       admin.disableTable(originalTableName);
     }
 
-    admin.restoreSnapshot(snapshotName);
+    admin.restoreSnapshot(snapshotNameAsString);
     admin.enableTable(originalTableName);
 
     // verify that the descrption is reverted
-    Table original = UTIL.getConnection().getTable(originalTableName);
-    try {
-      assertTrue(originalTableDescriptor.equals(admin.getTableDescriptor(originalTableName)));
-      assertTrue(originalTableDescriptor.equals(original.getTableDescriptor()));
-    } finally {
-      original.close();
+    try (Table original = UTIL.getConnection().getTable(originalTableName)) {
+      assertEquals(originalTableDescriptor,
+        new HTableDescriptor(admin.getDescriptor(originalTableName)));
+      assertEquals(originalTableDescriptor, new HTableDescriptor(original.getDescriptor()));
     }
   }
 }
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 f58b947..589b31d 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
@@ -29,6 +29,7 @@ import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 import java.util.UUID;
+import java.util.regex.Pattern;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -152,9 +153,9 @@ public class TestSnapshotTemporaryDirectory {
       throws IOException, InterruptedException {
     long tid = System.currentTimeMillis();
     TableName tableName = TableName.valueOf("testtb-" + tid);
-    byte[] emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
-    byte[] snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
-    byte[] snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
+    String emptySnapshot = "emptySnaptb-" + tid;
+    String snapshotName0 = "snaptb0-" + tid;
+    String snapshotName1 = "snaptb1-" + tid;
     int snapshot0Rows;
     int snapshot1Rows;
 
@@ -163,7 +164,7 @@ public class TestSnapshotTemporaryDirectory {
     admin.disableTable(tableName);
 
     // take an empty snapshot
-    takeSnapshot(tableName, Bytes.toString(emptySnapshot), true);
+    takeSnapshot(tableName, emptySnapshot, true);
 
     // enable table and insert data
     admin.enableTable(tableName);
@@ -174,7 +175,7 @@ public class TestSnapshotTemporaryDirectory {
     admin.disableTable(tableName);
 
     // take a snapshot
-    takeSnapshot(tableName, Bytes.toString(snapshotName0), true);
+    takeSnapshot(tableName, snapshotName0, true);
 
     // enable table and insert more data
     admin.enableTable(tableName);
@@ -185,7 +186,7 @@ public class TestSnapshotTemporaryDirectory {
 
     SnapshotTestingUtils.verifyRowCount(UTIL, tableName, snapshot1Rows);
     admin.disableTable(tableName);
-    takeSnapshot(tableName, Bytes.toString(snapshotName1), true);
+    takeSnapshot(tableName, snapshotName1, true);
 
     // Restore from snapshot-0
     admin.restoreSnapshot(snapshotName0);
@@ -218,9 +219,9 @@ public class TestSnapshotTemporaryDirectory {
       throws IOException, InterruptedException {
     long tid = System.currentTimeMillis();
     TableName tableName = TableName.valueOf("testtb-" + tid);
-    byte[] emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
-    byte[] snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
-    byte[] snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
+    String emptySnapshot = "emptySnaptb-" + tid;
+    String snapshotName0 = "snaptb0-" + tid;
+    String snapshotName1 = "snaptb1-" + tid;
     int snapshot0Rows;
     int snapshot1Rows;
 
@@ -228,7 +229,7 @@ public class TestSnapshotTemporaryDirectory {
     SnapshotTestingUtils.createTable(UTIL, tableName, getNumReplicas(), TEST_FAM);
 
     // take an empty snapshot
-    takeSnapshot(tableName, Bytes.toString(emptySnapshot), false);
+    takeSnapshot(tableName, emptySnapshot, false);
 
     // Insert data
     SnapshotTestingUtils.loadData(UTIL, tableName, 500, TEST_FAM);
@@ -237,7 +238,7 @@ public class TestSnapshotTemporaryDirectory {
     }
 
     // take a snapshot
-    takeSnapshot(tableName, Bytes.toString(snapshotName0), false);
+    takeSnapshot(tableName, snapshotName0, false);
 
     // Insert more data
     SnapshotTestingUtils.loadData(UTIL, tableName, 500, TEST_FAM);
@@ -246,7 +247,7 @@ public class TestSnapshotTemporaryDirectory {
     }
 
     SnapshotTestingUtils.verifyRowCount(UTIL, tableName, snapshot1Rows);
-    takeSnapshot(tableName, Bytes.toString(snapshotName1), false);
+    takeSnapshot(tableName, snapshotName1, false);
 
     // Restore from snapshot-0
     admin.disableTable(tableName);
@@ -416,12 +417,11 @@ public class TestSnapshotTemporaryDirectory {
   }
 
   // Ensures that the snapshot is transferred to the proper completed snapshot directory
-  @Test(timeout = 180000) public void testEnsureTemporaryDirectoryTransfer() throws Exception {
-    Admin admin = null;
+  @Test(timeout = 180000)
+  public void testEnsureTemporaryDirectoryTransfer() throws Exception {
+    Admin admin = UTIL.getAdmin();
     TableName tableName2 = TableName.valueOf("testListTableSnapshots");
     try {
-      admin = UTIL.getHBaseAdmin();
-
       HTableDescriptor htd = new HTableDescriptor(tableName2);
       UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
 
@@ -437,7 +437,8 @@ public class TestSnapshotTemporaryDirectory {
       takeSnapshot(TABLE_NAME, table2Snapshot1, false);
       LOG.debug("Table2Snapshot1 completed.");
 
-      List<SnapshotDescription> listTableSnapshots = admin.listTableSnapshots("test.*", ".*");
+      List<SnapshotDescription> listTableSnapshots =
+        admin.listTableSnapshots(Pattern.compile("test.*"), Pattern.compile(".*"));
       List<String> listTableSnapshotNames = new ArrayList<String>();
       assertEquals(3, listTableSnapshots.size());
       for (SnapshotDescription s : listTableSnapshots) {
@@ -447,16 +448,14 @@ public class TestSnapshotTemporaryDirectory {
       assertTrue(listTableSnapshotNames.contains(table1Snapshot2));
       assertTrue(listTableSnapshotNames.contains(table2Snapshot1));
     } finally {
-      if (admin != null) {
-        try {
-          admin.deleteSnapshots("Table.*");
-        } catch (SnapshotDoesNotExistException ignore) {
-        }
-        if (admin.tableExists(tableName2)) {
-          UTIL.deleteTable(tableName2);
-        }
-        admin.close();
+      try {
+        admin.deleteSnapshots(Pattern.compile("Table.*"));
+      } catch (SnapshotDoesNotExistException ignore) {
+      }
+      if (admin.tableExists(tableName2)) {
+        UTIL.deleteTable(tableName2);
       }
+      admin.close();
     }
   }
 
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 64392eb..3e40b6f 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
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -29,7 +28,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -84,20 +82,18 @@ public class TestSplitOrMergeStatus {
 
     Admin admin = TEST_UTIL.getAdmin();
     initSwitchStatus(admin);
-    boolean[] results = admin.setSplitOrMergeEnabled(false, false, MasterSwitchType.SPLIT);
-    assertEquals(1, results.length);
-    assertTrue(results[0]);
+    boolean result = admin.splitSwitch(false, false);
+    assertTrue(result);
     admin.split(t.getName());
-    int count = admin.getTableRegions(tableName).size();
+    int count = admin.getRegions(tableName).size();
     assertTrue(originalCount == count);
-    results = admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.SPLIT);
-    assertEquals(1, results.length);
-    assertFalse(results[0]);
+    result = admin.splitSwitch(true, false);
+    assertFalse(result);
     admin.split(t.getName());
-    while ((count = admin.getTableRegions(tableName).size()) == originalCount) {
+    while ((count = admin.getRegions(tableName).size()) == originalCount) {
       Threads.sleep(1);
     }
-    count = admin.getTableRegions(tableName).size();
+    count = admin.getRegions(tableName).size();
     assertTrue(originalCount < count);
     admin.close();
   }
@@ -110,21 +106,20 @@ public class TestSplitOrMergeStatus {
     TEST_UTIL.loadTable(t, FAMILY, false);
 
     Admin admin = TEST_UTIL.getAdmin();
-    int originalCount = admin.getTableRegions(tableName).size();
+    int originalCount = admin.getRegions(tableName).size();
     initSwitchStatus(admin);
     admin.split(t.getName());
     int postSplitCount = -1;
-    while ((postSplitCount = admin.getTableRegions(tableName).size()) == originalCount) {
+    while ((postSplitCount = admin.getRegions(tableName).size()) == originalCount) {
       Threads.sleep(1);
     }
     assertTrue("originalCount=" + originalCount + ", newCount=" + postSplitCount,
         originalCount != postSplitCount);
 
     // Merge switch is off so merge should NOT succeed.
-    boolean[] results = admin.setSplitOrMergeEnabled(false, false, MasterSwitchType.MERGE);
-    assertEquals(1, results.length);
-    assertTrue(results[0]);
-    List<HRegionInfo> regions = admin.getTableRegions(t.getName());
+    boolean result = admin.mergeSwitch(false, false);
+    assertTrue(result);
+    List<RegionInfo> regions = admin.getRegions(t.getName());
     assertTrue(regions.size() > 1);
     Future<?> f = admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
       regions.get(1).getEncodedNameAsBytes(), true);
@@ -134,17 +129,16 @@ public class TestSplitOrMergeStatus {
     } catch (ExecutionException ee) {
       // Expected.
     }
-    int count = admin.getTableRegions(tableName).size();
+    int count = admin.getRegions(tableName).size();
     assertTrue("newCount=" + postSplitCount + ", count=" + count, postSplitCount == count);
 
-    results = admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.MERGE);
-    regions = admin.getTableRegions(t.getName());
-    assertEquals(1, results.length);
-    assertFalse(results[0]);
+    result = admin.mergeSwitch(true, false);
+    regions = admin.getRegions(t.getName());
+    assertFalse(result);
     f = admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
       regions.get(1).getEncodedNameAsBytes(), true);
     f.get(10, TimeUnit.SECONDS);
-    count = admin.getTableRegions(tableName).size();
+    count = admin.getRegions(tableName).size();
     assertTrue((postSplitCount / 2 /*Merge*/) == count);
     admin.close();
   }
@@ -152,24 +146,22 @@ public class TestSplitOrMergeStatus {
   @Test
   public void testMultiSwitches() throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
-    boolean[] switches = admin.setSplitOrMergeEnabled(false, false,
-      MasterSwitchType.SPLIT, MasterSwitchType.MERGE);
-    for (boolean s : switches){
-      assertTrue(s);
-    }
-    assertFalse(admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT));
-    assertFalse(admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE));
+    assertTrue(admin.splitSwitch(false, false));
+    assertTrue(admin.mergeSwitch(false, false));
+
+    assertFalse(admin.isSplitEnabled());
+    assertFalse(admin.isMergeEnabled());
     admin.close();
   }
 
   private void initSwitchStatus(Admin admin) throws IOException {
-    if (!admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT)) {
-      admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.SPLIT);
+    if (!admin.isSplitEnabled()) {
+      admin.splitSwitch(true, false);
     }
-    if (!admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE)) {
-      admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.MERGE);
+    if (!admin.isMergeEnabled()) {
+      admin.mergeSwitch(true, false);
     }
-    assertTrue(admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT));
-    assertTrue(admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE));
+    assertTrue(admin.isSplitEnabled());
+    assertTrue(admin.isMergeEnabled());
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
index 716b1dc..fd4cc51 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
@@ -114,8 +114,8 @@ public class TestTableFavoredNodes {
   public void setup() throws IOException {
     fnm = TEST_UTIL.getMiniHBaseCluster().getMaster().getFavoredNodesManager();
     admin = TEST_UTIL.getAdmin();
-    admin.setBalancerRunning(false, true);
-    admin.enableCatalogJanitor(false);
+    admin.balancerSwitch(false, true);
+    admin.catalogJanitorSwitch(false);
   }
 
   /*
@@ -130,7 +130,7 @@ public class TestTableFavoredNodes {
     // All regions should have favored nodes
     checkIfFavoredNodeInformationIsCorrect(tableName);
 
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
 
     TEST_UTIL.deleteTable(tableName);
 
@@ -149,13 +149,13 @@ public class TestTableFavoredNodes {
     // All regions should have favored nodes
     checkIfFavoredNodeInformationIsCorrect(tableName);
 
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     TEST_UTIL.truncateTable(tableName, true);
 
     checkNoFNForDeletedTable(regions);
     checkIfFavoredNodeInformationIsCorrect(tableName);
 
-    regions = admin.getTableRegions(tableName);
+    regions = admin.getRegions(tableName);
     TEST_UTIL.truncateTable(tableName, false);
     checkNoFNForDeletedTable(regions);
 
@@ -170,7 +170,7 @@ public class TestTableFavoredNodes {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     Table t = TEST_UTIL.createTable(tableName, Bytes.toBytes("f"), splitKeys);
     TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
-    final int numberOfRegions = admin.getTableRegions(t.getName()).size();
+    final int numberOfRegions = admin.getRegions(t.getName()).size();
 
     checkIfFavoredNodeInformationIsCorrect(tableName);
 
@@ -211,14 +211,14 @@ public class TestTableFavoredNodes {
 
     // Major compact table and run catalog janitor. Parent's FN should be removed
     TEST_UTIL.getMiniHBaseCluster().compact(tableName, true);
-    admin.runCatalogScan();
+    admin.runCatalogJanitor();
     // Catalog cleanup is async. Wait on procedure to finish up.
     ProcedureTestingUtility.waitAllProcedures(
         TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor());
     // assertEquals("Parent region should have been cleaned", 1, admin.runCatalogScan());
     assertNull("Parent FN should be null", fnm.getFavoredNodes(parent));
 
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     // Split and Table Disable interfere with each other around region replicas
     // TODO. Meantime pause a few seconds.
     Threads.sleep(2000);
@@ -266,22 +266,22 @@ public class TestTableFavoredNodes {
 
     // Major compact table and run catalog janitor. Parent FN should be removed
     TEST_UTIL.getMiniHBaseCluster().compact(tableName, true);
-    assertEquals("Merge parents should have been cleaned", 1, admin.runCatalogScan());
+    assertEquals("Merge parents should have been cleaned", 1, admin.runCatalogJanitor());
     // Catalog cleanup is async. Wait on procedure to finish up.
     ProcedureTestingUtility.waitAllProcedures(
         TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor());
     assertNull("Parent FN should be null", fnm.getFavoredNodes(regionA));
     assertNull("Parent FN should be null", fnm.getFavoredNodes(regionB));
 
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
 
     TEST_UTIL.deleteTable(tableName);
 
     checkNoFNForDeletedTable(regions);
   }
 
-  private void checkNoFNForDeletedTable(List<HRegionInfo> regions) {
-    for (HRegionInfo region : regions) {
+  private void checkNoFNForDeletedTable(List<RegionInfo> regions) {
+    for (RegionInfo region : regions) {
       LOG.info("Testing if FN data for " + region);
       assertNull("FN not null for deleted table's region: " + region, fnm.getFavoredNodes(region));
     }
@@ -362,8 +362,8 @@ public class TestTableFavoredNodes {
 
     for (TableName sysTable :
         admin.listTableNamesByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
-      List<HRegionInfo> regions = admin.getTableRegions(sysTable);
-      for (HRegionInfo region : regions) {
+      List<RegionInfo> regions = admin.getRegions(sysTable);
+      for (RegionInfo region : regions) {
         assertNull("FN should be null for sys region", fnm.getFavoredNodes(region));
       }
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java
index e999f8c..720ce2d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@@ -65,7 +64,7 @@ public class TestTableSnapshotScanner {
   public static void blockUntilSplitFinished(HBaseTestingUtility util, TableName tableName,
       int expectedRegionSize) throws Exception {
     for (int i = 0; i < 100; i++) {
-      List<HRegionInfo> hRegionInfoList = util.getAdmin().getTableRegions(tableName);
+      List<RegionInfo> hRegionInfoList = util.getAdmin().getRegions(tableName);
       if (hRegionInfoList.size() >= expectedRegionSize) {
         break;
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
index 0ff757e..46536c1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
@@ -93,15 +93,15 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
   @Test
   public void disableNotFullReplication() throws Exception {
-    HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
+    HTableDescriptor table = new HTableDescriptor(admin2.getDescriptor(tableName));
     HColumnDescriptor f = new HColumnDescriptor("notReplicatedFamily");
     table.addFamily(f);
     admin1.disableTable(tableName);
-    admin1.modifyTable(tableName, table);
+    admin1.modifyTable(table);
     admin1.enableTable(tableName);
 
     admin1.disableTableReplication(tableName);
-    table = admin1.getTableDescriptor(tableName);
+    table = new HTableDescriptor(admin1.getDescriptor(tableName));
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_LOCAL, fam.getScope());
     }
@@ -121,20 +121,20 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
   @Test
   public void testEnableReplicationWhenReplicationNotEnabled() throws Exception {
-    HTableDescriptor table = new HTableDescriptor(admin1.getTableDescriptor(tableName));
+    HTableDescriptor table = new HTableDescriptor(admin1.getDescriptor(tableName));
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       fam.setScope(HConstants.REPLICATION_SCOPE_LOCAL);
     }
     admin1.disableTable(tableName);
-    admin1.modifyTable(tableName, table);
+    admin1.modifyTable(table);
     admin1.enableTable(tableName);
 
     admin2.disableTable(tableName);
-    admin2.modifyTable(tableName, table);
+    admin2.modifyTable(table);
     admin2.enableTable(tableName);
 
     admin1.enableTableReplication(tableName);
-    table = admin1.getTableDescriptor(tableName);
+    table = new HTableDescriptor(admin1.getDescriptor(tableName));
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_GLOBAL, fam.getScope());
     }
@@ -142,11 +142,11 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
   @Test
   public void testEnableReplicationWhenTableDescriptorIsNotSameInClusters() throws Exception {
-    HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
+    HTableDescriptor table = new HTableDescriptor(admin2.getDescriptor(tableName));
     HColumnDescriptor f = new HColumnDescriptor("newFamily");
     table.addFamily(f);
     admin2.disableTable(tableName);
-    admin2.modifyTable(tableName, table);
+    admin2.modifyTable(table);
     admin2.enableTable(tableName);
 
     try {
@@ -156,10 +156,10 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
     }
     admin1.disableTable(tableName);
-    admin1.modifyTable(tableName, table);
+    admin1.modifyTable(table);
     admin1.enableTable(tableName);
     admin1.enableTableReplication(tableName);
-    table = admin1.getTableDescriptor(tableName);
+    table = new HTableDescriptor(admin1.getDescriptor(tableName));
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_GLOBAL, fam.getScope());
     }
@@ -171,12 +171,12 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
   @Test
   public void testDisableAndEnableReplication() throws Exception {
     admin1.disableTableReplication(tableName);
-    HTableDescriptor table = admin1.getTableDescriptor(tableName);
+    HTableDescriptor table = new HTableDescriptor(admin1.getDescriptor(tableName));
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_LOCAL, fam.getScope());
     }
     admin1.enableTableReplication(tableName);
-    table = admin1.getTableDescriptor(tableName);
+    table = new HTableDescriptor(admin1.getDescriptor(tableName));
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_GLOBAL, fam.getScope());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
index 24f74d8..473da0b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
@@ -286,7 +286,7 @@ public class TestCoprocessorMetrics {
   public void setup() throws IOException {
     try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
          Admin admin = connection.getAdmin()) {
-      for (HTableDescriptor htd : admin.listTables()) {
+      for (TableDescriptor htd : admin.listTableDescriptors()) {
         UTIL.deleteTable(htd.getTableName());
       }
     }
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 58f4b9b..178429d 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
@@ -1256,11 +1256,11 @@ public class TestMasterObserver {
   }
 
   private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
-  private static byte[] TEST_SNAPSHOT = Bytes.toBytes("observed_snapshot");
+  private static String TEST_SNAPSHOT = "observed_snapshot";
   private static TableName TEST_CLONE = TableName.valueOf("observed_clone");
   private static byte[] TEST_FAMILY = Bytes.toBytes("fam1");
-  private static byte[] TEST_FAMILY2 = Bytes.toBytes("fam2");
-  @Rule public TestName name = new TestName();
+  @Rule
+  public TestName name = new TestName();
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
@@ -1507,10 +1507,10 @@ public class TestMasterObserver {
 
   private void modifyTableSync(Admin admin, TableName tableName, HTableDescriptor htd)
       throws IOException {
-    admin.modifyTable(tableName, htd);
+    admin.modifyTable(htd);
     //wait until modify table finishes
     for (int t = 0; t < 100; t++) { //10 sec timeout
-      HTableDescriptor td = admin.getTableDescriptor(htd.getTableName());
+      HTableDescriptor td = new HTableDescriptor(admin.getDescriptor(htd.getTableName()));
       if (td.equals(htd)) {
         break;
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java
index b607ca7..ac46d9b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java
@@ -22,9 +22,9 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -80,7 +80,7 @@ public class TestLoadAndSwitchEncodeOnDisk extends
     compression = Compression.Algorithm.GZ; // used for table setup
     super.loadTest();
 
-    HColumnDescriptor hcd = getColumnDesc(admin);
+    ColumnFamilyDescriptor hcd = getColumnDesc(admin);
     System.err.println("\nDisabling encode-on-disk. Old column descriptor: " + hcd + "\n");
     Table t = TEST_UTIL.getConnection().getTable(TABLE);
     assertAllOnLine(t);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
index fcd1638..78911c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -91,13 +92,14 @@ public class TestCatalogJanitorInMemoryStates {
    * Test clearing a split parent from memory.
    */
   @Test
-  public void testInMemoryParentCleanup() throws IOException, InterruptedException {
+  public void testInMemoryParentCleanup()
+      throws IOException, InterruptedException, ExecutionException {
     final AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
     final ServerManager sm = TEST_UTIL.getHBaseCluster().getMaster().getServerManager();
     final CatalogJanitor janitor = TEST_UTIL.getHBaseCluster().getMaster().getCatalogJanitor();
 
     Admin admin = TEST_UTIL.getAdmin();
-    admin.enableCatalogJanitor(false);
+    admin.catalogJanitorSwitch(false);
 
     final TableName tableName = TableName.valueOf(name.getMethodName());
     Table t = TEST_UTIL.createTable(tableName, FAMILY);
@@ -129,19 +131,18 @@ public class TestCatalogJanitorInMemoryStates {
 
   }
 
-  /*
- * Splits a region
- * @param t Region to split.
- * @return List of region locations
- * @throws IOException, InterruptedException
- */
+  /**
+   * Splits a region
+   * @param t Region to split.
+   * @return List of region locations
+   */
   private List<HRegionLocation> splitRegion(final RegionInfo r)
-      throws IOException, InterruptedException {
+      throws IOException, InterruptedException, ExecutionException {
     List<HRegionLocation> locations = new ArrayList<>();
     // Split this table in two.
     Admin admin = TEST_UTIL.getAdmin();
     Connection connection = TEST_UTIL.getConnection();
-    admin.splitRegion(r.getEncodedNameAsBytes());
+    admin.splitRegionAsync(r.getEncodedNameAsBytes(), null).get();
     admin.close();
     PairOfSameType<RegionInfo> regions = waitOnDaughters(r);
     if (regions != null) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java
index 80f0fb6..93cd4f4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java
@@ -25,8 +25,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -147,7 +147,7 @@ public class TestMasterBalanceThrottling {
       Thread.sleep(100);
     }
     HRegionServer biasedServer = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
-    for (HRegionInfo regionInfo : TEST_UTIL.getAdmin().getTableRegions(tableName)) {
+    for (RegionInfo regionInfo : TEST_UTIL.getAdmin().getRegions(tableName)) {
       master.move(regionInfo.getEncodedNameAsBytes(),
         Bytes.toBytes(biasedServer.getServerName().getServerName()));
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
index 4d5c946..44cf1cc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
@@ -219,9 +219,9 @@ public class TestMasterOperationsForRegionReplicas {
       assert(ADMIN.isTableDisabled(tableName));
       //increase the replica
       desc.setRegionReplication(numReplica + 1);
-      ADMIN.modifyTable(tableName, desc);
+      ADMIN.modifyTable(desc);
       ADMIN.enableTable(tableName);
-      LOG.info(ADMIN.getTableDescriptor(tableName).toString());
+      LOG.info(ADMIN.getDescriptor(tableName).toString());
       assert(ADMIN.isTableEnabled(tableName));
       List<RegionInfo> regions = TEST_UTIL.getMiniHBaseCluster().getMaster().
           getAssignmentManager().getRegionStates().getRegionsOfTable(tableName);
@@ -231,7 +231,7 @@ public class TestMasterOperationsForRegionReplicas {
       //decrease the replica(earlier, table was modified to have a replica count of numReplica + 1)
       ADMIN.disableTable(tableName);
       desc.setRegionReplication(numReplica);
-      ADMIN.modifyTable(tableName, desc);
+      ADMIN.modifyTable(desc);
       ADMIN.enableTable(tableName);
       assert(ADMIN.isTableEnabled(tableName));
       regions = TEST_UTIL.getMiniHBaseCluster().getMaster()
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
index cf80d8e..9f377a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
@@ -118,11 +120,9 @@ public class TestMasterStatusServlet {
   }
 
   private void setupMockTables() throws IOException {
-    HTableDescriptor tables[] = new HTableDescriptor[] {
-        new HTableDescriptor(TableName.valueOf("foo")),
-        new HTableDescriptor(TableName.valueOf("bar"))
-    };
-    Mockito.doReturn(tables).when(admin).listTables();
+    List<TableDescriptor> tables = Arrays.asList(new HTableDescriptor(TableName.valueOf("foo")),
+      new HTableDescriptor(TableName.valueOf("bar")));
+    Mockito.doReturn(tables).when(admin).listTableDescriptors();
   }
 
   @Test
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 c4055fe..839d611 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().getTableRegions(TEST_TABLE)) {
+    for (RegionInfo hri: UTIL.getHBaseAdmin().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/TestRogueRSAssignment.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRogueRSAssignment.java
index c88e583..70388ef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRogueRSAssignment.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRogueRSAssignment.java
@@ -26,13 +26,13 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -125,7 +125,7 @@ public class TestRogueRSAssignment {
   @Before
   public void setup() throws IOException {
     // Turn off balancer
-    admin.setBalancerRunning(false, true);
+    admin.balancerSwitch(false, true);
   }
 
   @After
@@ -135,7 +135,7 @@ public class TestRogueRSAssignment {
       UTIL.deleteTable(td.getTableName());
     }
     // Turn on balancer
-    admin.setBalancerRunning(true, false);
+    admin.balancerSwitch(true, false);
   }
 
   /**
@@ -146,7 +146,7 @@ public class TestRogueRSAssignment {
   public void testReportRSWithWrongRegion() throws Exception {
     final TableName tableName = TableName.valueOf(this.name.getMethodName());
 
-    List<HRegionInfo> tableRegions = createTable(tableName);
+    List<RegionInfo> tableRegions = createTable(tableName);
 
     final ServerName sn = ServerName.parseVersionedServerName(
         ServerName.valueOf("1.example.org", 1, System.currentTimeMillis()).getVersionedBytes());
@@ -164,7 +164,7 @@ public class TestRogueRSAssignment {
   }
 
   private RegionServerStatusProtos.RegionServerReportRequest.Builder
-      makeRSReportRequestWithRegions(final ServerName sn, HRegionInfo... regions) {
+      makeRSReportRequestWithRegions(final ServerName sn, RegionInfo... regions) {
     ClusterStatusProtos.ServerLoad.Builder sl = ClusterStatusProtos.ServerLoad.newBuilder();
     for (int i = 0; i < regions.length; i++) {
       HBaseProtos.RegionSpecifier.Builder rs = HBaseProtos.RegionSpecifier.newBuilder();
@@ -182,7 +182,7 @@ public class TestRogueRSAssignment {
               .setLoad(sl);
   }
 
-  private List<HRegionInfo> createTable(final TableName tableName) throws Exception {
+  private List<RegionInfo> createTable(final TableName tableName) throws Exception {
     TableDescriptorBuilder tdBuilder = TableDescriptorBuilder.newBuilder(tableName);
     tdBuilder.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build());
 
@@ -194,10 +194,10 @@ public class TestRogueRSAssignment {
     return assertRegionCount(tableName, initialRegionCount);
   }
 
-  private List<HRegionInfo> assertRegionCount(final TableName tableName, final int nregions)
+  private List<RegionInfo> assertRegionCount(final TableName tableName, final int nregions)
       throws Exception {
     UTIL.waitUntilNoRegionsInTransition();
-    List<HRegionInfo> tableRegions = admin.getTableRegions(tableName);
+    List<RegionInfo> tableRegions = admin.getRegions(tableName);
     assertEquals(nregions, tableRegions.size());
     return tableRegions;
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java
index b175bd7..612b7ef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.master.balancer;
 
 import static org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper.FAVORED_NODES_NUM;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 
 import java.util.List;
@@ -29,11 +28,11 @@ 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.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -82,7 +81,7 @@ public class TestFavoredNodeTableImport {
       Threads.sleep(1);
     }
     Admin admin = UTIL.getAdmin();
-    admin.setBalancerRunning(false, true);
+    admin.balancerSwitch(false, true);
 
     String tableName = "testFNImport";
     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
@@ -107,8 +106,8 @@ public class TestFavoredNodeTableImport {
 
     FavoredNodesManager fnm = UTIL.getHBaseCluster().getMaster().getFavoredNodesManager();
 
-    List<HRegionInfo> regionsOfTable = admin.getTableRegions(TableName.valueOf(tableName));
-    for (HRegionInfo rInfo : regionsOfTable) {
+    List<RegionInfo> regionsOfTable = admin.getRegions(TableName.valueOf(tableName));
+    for (RegionInfo rInfo : regionsOfTable) {
       Set<ServerName> favNodes = Sets.newHashSet(fnm.getFavoredNodes(rInfo));
       assertNotNull(favNodes);
       assertEquals("Required no of favored nodes not found.", FAVORED_NODES_NUM, favNodes.size());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
index 0b40ae0..6e1c84d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
@@ -109,7 +109,7 @@ public class TestFavoredStochasticBalancerPickers extends BalancerTestBase {
     TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(120*1000);
     cluster = TEST_UTIL.getMiniHBaseCluster();
     admin = TEST_UTIL.getAdmin();
-    admin.setBalancerRunning(false, true);
+    admin.balancerSwitch(false, true);
   }
 
   @After
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
index 4bc566c..494fca3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
@@ -104,7 +104,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     cluster = TEST_UTIL.getMiniHBaseCluster();
     master = TEST_UTIL.getMiniHBaseCluster().getMaster();
     admin = TEST_UTIL.getAdmin();
-    admin.setBalancerRunning(false, true);
+    admin.balancerSwitch(false, true);
   }
 
   @After
@@ -130,8 +130,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
     // Now try to run balance, and verify no regions are moved to the 2 region servers recently
     // started.
-    admin.setBalancerRunning(true, true);
-    assertTrue("Balancer did not run", admin.balancer());
+    admin.balancerSwitch(true, true);
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitUntilNoRegionsInTransition(120000);
 
     List<RegionInfo> hris = admin.getRegions(rs1.getRegionServer().getServerName());
@@ -159,7 +159,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
     LoadBalancer balancer = master.getLoadBalancer();
     List<RegionInfo> regions = admin.getRegions(tableName);
-    regions.addAll(admin.getTableRegions(TableName.META_TABLE_NAME));
+    regions.addAll(admin.getRegions(TableName.META_TABLE_NAME));
     List<ServerName> servers = Lists.newArrayList(
       admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet());
     Map<ServerName, List<RegionInfo>> map = balancer.roundRobinAssignment(regions, servers);
@@ -219,7 +219,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.createTable(desc);
     TEST_UTIL.waitTableAvailable(desc.getTableName());
 
-    RegionInfo hri = admin.getTableRegions(TableName.valueOf(tableName)).get(0);
+    RegionInfo hri = admin.getRegions(TableName.valueOf(tableName)).get(0);
 
     FavoredNodesManager fnm = master.getFavoredNodesManager();
     fnm.deleteFavoredNodesForRegions(Lists.newArrayList(hri));
@@ -250,7 +250,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
-    final RegionInfo region = admin.getTableRegions(tableName).get(0);
+    final RegionInfo region = admin.getRegions(tableName).get(0);
     LOG.info("Region thats supposed to be in transition: " + region);
     FavoredNodesManager fnm = master.getFavoredNodesManager();
     List<ServerName> currentFN = fnm.getFavoredNodes(region);
@@ -259,10 +259,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     fnm.deleteFavoredNodesForRegions(Lists.newArrayList(region));
 
     RegionStates regionStates = master.getAssignmentManager().getRegionStates();
-    admin.setBalancerRunning(true, true);
+    admin.balancerSwitch(true, true);
 
     // Balancer should unassign the region
-    assertTrue("Balancer did not run", admin.balancer());
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitUntilNoRegionsInTransition();
 
     admin.assign(region.getEncodedNameAsBytes());
@@ -273,7 +273,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     assertEquals("Expected number of FN not present",
       FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, currentFN.size());
 
-    assertTrue("Balancer did not run", admin.balancer());
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
 
     checkFavoredNodeAssignments(tableName, fnm, regionStates);
@@ -288,7 +288,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
-    final RegionInfo misplacedRegion = admin.getTableRegions(tableName).get(0);
+    final RegionInfo misplacedRegion = admin.getRegions(tableName).get(0);
     FavoredNodesManager fnm = master.getFavoredNodesManager();
     List<ServerName> currentFN = fnm.getFavoredNodes(misplacedRegion);
     assertNotNull(currentFN);
@@ -314,8 +314,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     final ServerName current = regionStates.getRegionServerOfRegion(misplacedRegion);
     assertNull("Misplaced region is still hosted on favored node, not expected.",
         FavoredNodesPlan.getFavoredServerPosition(fnm.getFavoredNodes(misplacedRegion), current));
-    admin.setBalancerRunning(true, true);
-    assertTrue("Balancer did not run", admin.balancer());
+    admin.balancerSwitch(true, true);
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitFor(120000, 30000, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
@@ -335,7 +335,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
-    final RegionInfo region = admin.getTableRegions(tableName).get(0);
+    final RegionInfo region = admin.getRegions(tableName).get(0);
     LOG.info("Region that's supposed to be in transition: " + region);
     FavoredNodesManager fnm = master.getFavoredNodesManager();
     List<ServerName> currentFN = fnm.getFavoredNodes(region);
@@ -356,9 +356,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
       }
     });
 
-    assertEquals("Not all regions are online", REGION_NUM, admin.getTableRegions(tableName).size());
-    admin.setBalancerRunning(true, true);
-    assertTrue("Balancer did not run", admin.balancer());
+    assertEquals("Not all regions are online", REGION_NUM, admin.getRegions(tableName).size());
+    admin.balancerSwitch(true, true);
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
 
     checkFavoredNodeAssignments(tableName, fnm, regionStates);
@@ -373,7 +373,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
-    final RegionInfo region = admin.getTableRegions(tableName).get(0);
+    final RegionInfo region = admin.getRegions(tableName).get(0);
     LOG.info("Region that's supposed to be in transition: " + region);
     FavoredNodesManager fnm = master.getFavoredNodesManager();
     List<ServerName> currentFN = fnm.getFavoredNodes(region);
@@ -417,10 +417,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
       admin.assign(regionInfo.getEncodedNameAsBytes());
     }
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
-    assertEquals("Not all regions are online", REGION_NUM, admin.getTableRegions(tableName).size());
+    assertEquals("Not all regions are online", REGION_NUM, admin.getRegions(tableName).size());
 
-    admin.setBalancerRunning(true, true);
-    assertTrue("Balancer did not run", admin.balancer());
+    admin.balancerSwitch(true, true);
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
 
     checkFavoredNodeAssignments(tableName, fnm, regionStates);
@@ -435,7 +435,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
-    final RegionInfo region = admin.getTableRegions(tableName).get(0);
+    final RegionInfo region = admin.getRegions(tableName).get(0);
     LOG.info("Region that's supposed to be in transition: " + region);
     FavoredNodesManager fnm = master.getFavoredNodesManager();
     List<ServerName> currentFN = fnm.getFavoredNodes(region);
@@ -511,10 +511,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
       admin.assign(regionInfo.getEncodedNameAsBytes());
     }
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
-    assertEquals("Not all regions are online", REGION_NUM, admin.getTableRegions(tableName).size());
+    assertEquals("Not all regions are online", REGION_NUM, admin.getRegions(tableName).size());
 
-    admin.setBalancerRunning(true, true);
-    assertTrue("Balancer did not run", admin.balancer());
+    admin.balancerSwitch(true, true);
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
 
     checkFavoredNodeAssignments(tableName, fnm, regionStates);
@@ -522,7 +522,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
   private void checkFavoredNodeAssignments(TableName tableName, FavoredNodesManager fnm,
       RegionStates regionStates) throws IOException {
-    for (RegionInfo hri : admin.getTableRegions(tableName)) {
+    for (RegionInfo hri : admin.getRegions(tableName)) {
       ServerName host = regionStates.getRegionServerOfRegion(hri);
       assertNotNull("Region: " + hri.getEncodedName() + " not on FN, current: " + host
               + " FN list: " + fnm.getFavoredNodes(hri),
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
index d04ef79..6860b7a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
@@ -25,9 +25,9 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -67,7 +67,7 @@ public class TestLockManager {
 
   private static String namespace = "namespace";
   private static TableName tableName = TableName.valueOf(namespace, "table");
-  private static HRegionInfo[] tableRegions;
+  private static RegionInfo[] tableRegions;
 
   private static void setupConf(Configuration conf) {
     conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
@@ -83,9 +83,9 @@ public class TestLockManager {
     UTIL.getAdmin().createNamespace(NamespaceDescriptor.create(namespace).build());
     UTIL.createTable(tableName, new byte[][]{Bytes.toBytes("fam")},
         new byte[][] {Bytes.toBytes("1")});
-    List<HRegionInfo> regions = UTIL.getAdmin().getTableRegions(tableName);
+    List<RegionInfo> regions = UTIL.getAdmin().getRegions(tableName);
     assert regions.size() > 0;
-    tableRegions = new HRegionInfo[regions.size()];
+    tableRegions = new RegionInfo[regions.size()];
     regions.toArray(tableRegions);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index 6f7f69e..537090a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -141,9 +141,9 @@ public class TestSimpleRegionNormalizerOnCluster {
       region.flush(true);
     }
 
-    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(TABLENAME));
+    HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
     htd.setNormalizationEnabled(true);
-    admin.modifyTable(TABLENAME, htd);
+    admin.modifyTable(htd);
 
     admin.flush(TABLENAME);
 
@@ -213,9 +213,9 @@ public class TestSimpleRegionNormalizerOnCluster {
       region.flush(true);
     }
 
-    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(tableName));
+    HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(tableName));
     htd.setNormalizationEnabled(true);
-    admin.modifyTable(tableName, htd);
+    admin.modifyTable(htd);
 
     admin.flush(tableName);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
index f4fa282..d90192a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
@@ -71,7 +71,7 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
     if (snapshot == null) {
       final TableName snapshotTableName = TableName.valueOf("testCloneSnapshot");
       long tid = System.currentTimeMillis();
-      final byte[] snapshotName = Bytes.toBytes("snapshot-" + tid);
+      final String snapshotName = "snapshot-" + tid;
 
       Admin admin = UTIL.getAdmin();
       // create Table
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java
index d5ec62d..91295c5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java
@@ -103,7 +103,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
   @Test
   public void deleteColumnFamilyWithMultipleRegions() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
-    HTableDescriptor beforehtd = admin.getTableDescriptor(TABLENAME);
+    HTableDescriptor beforehtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
 
     FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
 
@@ -150,7 +150,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
     admin.deleteColumnFamily(TABLENAME, Bytes.toBytes("cf2"));
 
     // 5 - Check if only 2 column families exist in the descriptor
-    HTableDescriptor afterhtd = admin.getTableDescriptor(TABLENAME);
+    HTableDescriptor afterhtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
     assertEquals(2, afterhtd.getColumnFamilyCount());
     HColumnDescriptor[] newFamilies = afterhtd.getColumnFamilies();
     assertTrue(newFamilies[0].getNameAsString().equals("cf1"));
@@ -181,7 +181,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
   @Test
   public void deleteColumnFamilyTwice() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
-    HTableDescriptor beforehtd = admin.getTableDescriptor(TABLENAME);
+    HTableDescriptor beforehtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
     String cfToDelete = "cf1";
 
     FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
index 5e574b5..ef65f2f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
@@ -205,7 +205,7 @@ public class TestMasterFailoverWithProcedures {
     UTIL.waitUntilAllRegionsAssigned(tableName);
 
     // validate the table regions and layout
-    regions = UTIL.getAdmin().getTableRegions(tableName).toArray(new RegionInfo[0]);
+    regions = UTIL.getAdmin().getRegions(tableName).toArray(new RegionInfo[0]);
     if (preserveSplits) {
       assertEquals(1 + splitKeys.length, regions.length);
     } else {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
index a2dccf8..8bd3958 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
@@ -73,7 +73,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     UTIL.getAdmin().disableTable(tableName);
 
     // Modify the table descriptor
-    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
 
     // Test 1: Modify 1 property
     long newMaxFileSize = htd.getMaxFileSize() * 2;
@@ -84,7 +84,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
         procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1));
 
-    HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(newMaxFileSize, currentHtd.getMaxFileSize());
 
     // Test 2: Modify multiple properties
@@ -97,7 +97,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
         procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
 
-    currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(newReadOnlyOption, currentHtd.isReadOnly());
     assertEquals(newMemStoreFlushSize, currentHtd.getMemStoreFlushSize());
   }
@@ -108,19 +108,19 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
 
     MasterProcedureTestingUtility.createTable(procExec, tableName, null, "cf1");
-    HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(1, currentHtd.getFamiliesKeys().size());
 
     // Test 1: Modify the table descriptor online
     String cf2 = "cf2";
-    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     htd.addFamily(new HColumnDescriptor(cf2));
 
     long procId = ProcedureTestingUtility.submitAndWait(
         procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
 
-    currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(2, currentHtd.getFamiliesKeys().size());
     assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf2)));
 
@@ -129,7 +129,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     ProcedureTestingUtility.waitNoProcedureRunning(procExec);
     String cf3 = "cf3";
     HTableDescriptor htd2 =
-        new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+        new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     htd2.addFamily(new HColumnDescriptor(cf3));
 
     long procId2 =
@@ -137,7 +137,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
           new ModifyTableProcedure(procExec.getEnvironment(), htd2));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
 
-    currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf3)));
     assertEquals(3, currentHtd.getFamiliesKeys().size());
   }
@@ -151,18 +151,18 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
 
     MasterProcedureTestingUtility.createTable(procExec, tableName, null, cf1, cf2, cf3);
-    HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(3, currentHtd.getFamiliesKeys().size());
 
     // Test 1: Modify the table descriptor
-    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     htd.removeFamily(Bytes.toBytes(cf2));
 
     long procId = ProcedureTestingUtility.submitAndWait(
         procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
 
-    currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(2, currentHtd.getFamiliesKeys().size());
     assertFalse(currentHtd.hasFamily(Bytes.toBytes(cf2)));
 
@@ -171,7 +171,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     ProcedureTestingUtility.waitNoProcedureRunning(procExec);
 
     HTableDescriptor htd2 =
-        new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+        new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     htd2.removeFamily(Bytes.toBytes(cf3));
     // Disable Sanity check
     htd2.setConfiguration("hbase.table.sanity.checks", Boolean.FALSE.toString());
@@ -181,13 +181,13 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
           new ModifyTableProcedure(procExec.getEnvironment(), htd2));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
 
-    currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(1, currentHtd.getFamiliesKeys().size());
     assertFalse(currentHtd.hasFamily(Bytes.toBytes(cf3)));
 
     //Removing the last family will fail
     HTableDescriptor htd3 =
-        new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+        new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     htd3.removeFamily(Bytes.toBytes(cf1));
     long procId3 =
         ProcedureTestingUtility.submitAndWait(procExec,
@@ -256,7 +256,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
 
     // Modify multiple properties of the table.
-    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     boolean newCompactionEnableOption = htd.isCompactionEnabled() ? false : true;
     htd.setCompactionEnabled(newCompactionEnableOption);
     htd.addFamily(new HColumnDescriptor(cf2));
@@ -270,7 +270,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     // Validate descriptor
-    HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(newCompactionEnableOption, currentHtd.isCompactionEnabled());
     assertEquals(2, currentHtd.getFamiliesKeys().size());
     assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf2)));
@@ -554,7 +554,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
 
       public void run() {
         try {
-          UTIL.getAdmin().modifyTable(tableName, htd);
+          UTIL.getAdmin().modifyTable(htd);
         } catch (Exception e) {
           if (e.getClass().equals(ConcurrentTableModificationException.class)) {
             this.exception = true;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
index 02f0257..c54aea4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
@@ -25,9 +25,9 @@ import java.util.Random;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -88,7 +88,7 @@ public class TestProcedureAdmin {
   public void tearDown() throws Exception {
     assertTrue("expected executor to be running", getMasterProcedureExecutor().isRunning());
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
-    for (HTableDescriptor htd: UTIL.getAdmin().listTables()) {
+    for (TableDescriptor htd: UTIL.getAdmin().listTableDescriptors()) {
       LOG.info("Tear down, remove table=" + htd.getTableName());
       UTIL.deleteTable(htd.getTableName());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
index e9e5ebe..0f7b7ca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
@@ -99,7 +99,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
 
   private void setupSnapshotAndUpdateTable() throws Exception {
     long tid = System.currentTimeMillis();
-    final byte[] snapshotName = Bytes.toBytes("snapshot-" + tid);
+    final String snapshotName = "snapshot-" + tid;
     Admin admin = UTIL.getAdmin();
     // create Table
     SnapshotTestingUtils.createTable(UTIL, snapshotTableName, getNumReplicas(), CF1, CF2);
@@ -108,7 +108,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
     SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF2, CF2);
     SnapshotTestingUtils.verifyRowCount(UTIL, snapshotTableName, rowCountCF1 + rowCountCF2);
 
-    snapshotHTD = admin.getTableDescriptor(snapshotTableName);
+    snapshotHTD = new HTableDescriptor(admin.getDescriptor(snapshotTableName));
 
     admin.disableTable(snapshotTableName);
     // take a snapshot
@@ -128,7 +128,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
     SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF3, CF3);
     SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF4, CF4);
     SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF1addition, CF1);
-    HTableDescriptor currentHTD = admin.getTableDescriptor(snapshotTableName);
+    HTableDescriptor currentHTD = new HTableDescriptor(admin.getDescriptor(snapshotTableName));
     assertTrue(currentHTD.hasFamily(CF1));
     assertFalse(currentHTD.hasFamily(CF2));
     assertTrue(currentHTD.hasFamily(CF3));
@@ -216,7 +216,8 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
     try {
       UTIL.getAdmin().enableTable(snapshotTableName);
 
-      HTableDescriptor currentHTD = UTIL.getAdmin().getTableDescriptor(snapshotTableName);
+      HTableDescriptor currentHTD =
+        new HTableDescriptor(UTIL.getAdmin().getDescriptor(snapshotTableName));
       assertTrue(currentHTD.hasFamily(CF1));
       assertTrue(currentHTD.hasFamily(CF2));
       assertFalse(currentHTD.hasFamily(CF3));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java
index f7cf640..bd7f26d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java
@@ -18,9 +18,11 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
+import static org.junit.Assert.assertTrue;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -31,8 +33,6 @@ import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.junit.Assert.assertTrue;
-
 public abstract class TestTableDDLProcedureBase {
   private static final Logger LOG = LoggerFactory.getLogger(TestTableDDLProcedureBase.class);
   protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
@@ -64,7 +64,7 @@ public abstract class TestTableDDLProcedureBase {
   @After
   public void tearDown() throws Exception {
     resetProcExecutorTestingKillFlag();
-    for (HTableDescriptor htd: UTIL.getAdmin().listTables()) {
+    for (TableDescriptor htd : UTIL.getAdmin().listTableDescriptors()) {
       LOG.info("Tear down, remove table=" + htd.getTableName());
       UTIL.deleteTable(htd.getTableName());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
index d3b13ad..11d1f19 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
@@ -101,7 +101,7 @@ public class TestTableDescriptorModificationFromClient {
       HTableDescriptor modifiedHtd = new HTableDescriptor(TABLE_NAME);
       modifiedHtd.addFamily(new HColumnDescriptor(FAMILY_0));
       modifiedHtd.addFamily(new HColumnDescriptor(FAMILY_1));
-      admin.modifyTable(TABLE_NAME, modifiedHtd);
+      admin.modifyTable(modifiedHtd);
       verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
     } finally {
       admin.deleteTable(TABLE_NAME);
@@ -178,7 +178,7 @@ public class TestTableDescriptorModificationFromClient {
       // Modify colymn family
       admin.modifyColumnFamily(TABLE_NAME, cfDescriptor);
 
-      HTableDescriptor htd = admin.getTableDescriptor(TABLE_NAME);
+      HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(TABLE_NAME));
       HColumnDescriptor hcfd = htd.getFamily(FAMILY_0);
       assertTrue(hcfd.getBlocksize() == newBlockSize);
     } finally {
@@ -267,12 +267,12 @@ public class TestTableDescriptorModificationFromClient {
     }
   }
 
-  private void verifyTableDescriptor(final TableName tableName,
-                                     final byte[]... families) throws IOException {
+  private void verifyTableDescriptor(final TableName tableName, final byte[]... families)
+      throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
 
     // Verify descriptor from master
-    HTableDescriptor htd = admin.getTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(tableName));
     verifyTableDescriptor(htd, tableName, families);
 
     // Verify descriptor from HDFS
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
index 86e07d2..130a6d3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
@@ -92,7 +92,6 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -268,13 +267,7 @@ public class TestMobCompactor {
 
     hcd1.setMobCompactPartitionPolicy(type);
     desc.modifyFamily(hcd1);
-    admin.modifyTable(tableName, desc);
-    Pair<Integer, Integer> st;
-
-    while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
-      LOG.debug(st.getFirst() + " regions left to update");
-      Thread.sleep(40);
-    }
+    admin.modifyTable(desc);
     LOG.info("alter status finished");
   }
 
@@ -439,7 +432,7 @@ public class TestMobCompactor {
     int rowNumPerRegion = count * rowNumPerFile;
 
     long tid = System.currentTimeMillis();
-    byte[] snapshotName1 = Bytes.toBytes("snaptb-" + tid);
+    String snapshotName1 = "snaptb-" + tid;
     // take a snapshot
     admin.snapshot(snapshotName1, tableName);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index a5d12b7..7b32b5c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -42,13 +42,11 @@ 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.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.CompactionState;
 import org.apache.hadoop.hbase.client.Connection;
@@ -83,8 +81,6 @@ import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.RetryCounter;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -130,7 +126,7 @@ public class TestNamespaceAuditor {
 
   @After
   public void cleanup() throws Exception, KeeperException {
-    for (HTableDescriptor table : ADMIN.listTables()) {
+    for (TableDescriptor table : ADMIN.listTableDescriptors()) {
       ADMIN.disableTable(table.getTableName());
       deleteTable(table.getTableName());
     }
@@ -714,8 +710,8 @@ public class TestNamespaceAuditor {
     String snapshot = "snapshot_testRestoreSnapshot";
     ADMIN.snapshot(snapshot, tableName1);
 
-    List<HRegionInfo> regions = ADMIN.getTableRegions(tableName1);
-    Collections.sort(regions);
+    List<RegionInfo> regions = ADMIN.getRegions(tableName1);
+    Collections.sort(regions, RegionInfo.COMPARATOR);
 
     ADMIN.split(tableName1, Bytes.toBytes("JJJ"));
     Thread.sleep(2000);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java
index 1b86b50..8ab9ec8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java
@@ -66,7 +66,7 @@ public class TestProcedureManager {
   public void testSimpleProcedureManager() throws IOException {
     Admin admin = util.getAdmin();
 
-    byte[] result = admin.execProcedureWithRet(SimpleMasterProcedureManager.SIMPLE_SIGNATURE,
+    byte[] result = admin.execProcedureWithReturn(SimpleMasterProcedureManager.SIMPLE_SIGNATURE,
         "mytest", new HashMap<>());
     assertArrayEquals("Incorrect return data from execProcedure",
         Bytes.toBytes(SimpleMasterProcedureManager.SIMPLE_DATA), result);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
index 2298c85..450c554 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
@@ -180,7 +180,7 @@ public class TestTablesWithQuotas {
     // Set up Admin to return null (match the implementation)
     Admin admin = mock(Admin.class);
     when(conn.getAdmin()).thenReturn(admin);
-    when(admin.getTableRegions(missingTable)).thenReturn(null);
+    when(admin.getRegions(missingTable)).thenReturn(null);
 
     QuotaObserverChore chore = mock(QuotaObserverChore.class);
     Map<RegionInfo,Long> regionUsage = new HashMap<>();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionFileNotFound.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionFileNotFound.java
index 22a888d..c19adfd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionFileNotFound.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionFileNotFound.java
@@ -126,12 +126,13 @@ public class TestCompactionFileNotFound {
       }
       table.put(putb);
       HRegion hr1 = (HRegion) util.getRSForFirstRegionInTable(TEST_TABLE)
-          .getRegionByEncodedName(admin.getTableRegions(TEST_TABLE).get(0).getEncodedName());
+          .getRegionByEncodedName(admin.getRegions(TEST_TABLE).get(0).getEncodedName());
       // Refresh store files post compaction, this should not open already compacted files
       hr1.refreshStoreFiles(true);
-      int numRegionsBeforeSplit = admin.getTableRegions(TEST_TABLE).size();
+      int numRegionsBeforeSplit = admin.getRegions(TEST_TABLE).size();
       // Check if we can successfully split after compaction
-      admin.splitRegion(admin.getTableRegions(TEST_TABLE).get(0).getEncodedNameAsBytes(), ROW_C);
+      admin.splitRegionAsync(admin.getRegions(TEST_TABLE).get(0).getEncodedNameAsBytes(), ROW_C)
+        .get();
       util.waitFor(20000, new Waiter.Predicate<Exception>() {
         @Override
         public boolean evaluate() throws Exception {
@@ -181,7 +182,7 @@ public class TestCompactionFileNotFound {
       }
       table.put(putb);
       HRegion hr1 = (HRegion) util.getRSForFirstRegionInTable(TEST_TABLE)
-          .getRegionByEncodedName(admin.getTableRegions(TEST_TABLE).get(0).getEncodedName());
+          .getRegionByEncodedName(admin.getRegions(TEST_TABLE).get(0).getEncodedName());
       // Refresh store files post compaction, this should not open already compacted files
       hr1.refreshStoreFiles(true);
       // Archive the store files and try another compaction to see if all is good
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 1418d6e..57ced95 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
@@ -262,10 +262,9 @@ public class TestEndToEndSplitTransaction {
 
           log("Initiating region split for:" + region.getRegionNameAsString());
           try {
-            admin.splitRegion(region.getRegionName(), splitPoint);
+            admin.splitRegionAsync(region.getRegionName(), splitPoint).get();
             // wait until the split is complete
             blockUntilRegionSplit(CONF, 50000, region.getRegionName(), true);
-
           } catch (NotServingRegionException ex) {
             // ignore
           }
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 9d9226e..ea93468 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
@@ -197,7 +197,7 @@ public class TestRegionMergeTransactionOnCluster {
   @Test
   public void testCleanMergeReference() throws Exception {
     LOG.info("Starting " + name.getMethodName());
-    ADMIN.enableCatalogJanitor(false);
+    ADMIN.catalogJanitorSwitch(false);
     try {
       final TableName tableName = TableName.valueOf(name.getMethodName());
       // Create table and load data.
@@ -278,7 +278,7 @@ public class TestRegionMergeTransactionOnCluster {
       // files of merging regions
       int cleaned = 0;
       while (cleaned == 0) {
-        cleaned = ADMIN.runCatalogScan();
+        cleaned = ADMIN.runCatalogJanitor();
         LOG.debug("catalog janitor returned " + cleaned);
         Thread.sleep(50);
         // Cleanup is async so wait till all procedures are done running.
@@ -297,7 +297,7 @@ public class TestRegionMergeTransactionOnCluster {
           HConstants.MERGEB_QUALIFIER) != null);
 
     } finally {
-      ADMIN.enableCatalogJanitor(true);
+      ADMIN.catalogJanitorSwitch(true);
     }
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
index 7190d84..7ab5dcc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.TableName;
 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.RegionInfo;
 import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -125,7 +125,7 @@ public class TestRegionOpen {
     } catch (IOException e) {
       LOG.info("Caught expected IOE due missing .regioninfo file, due: " + e.getMessage() + " skipping region open.");
       // We should only have 1 region online
-      List<HRegionInfo> regions = admin.getTableRegions(tableName);
+      List<RegionInfo> regions = admin.getRegions(tableName);
       LOG.info("Regions: " + regions);
       if (regions.size() != 1) {
         fail("Table " + tableName + " should have only one region, but got more: " + regions);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index 40c3d29..3778c20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@ -154,20 +154,6 @@ public class TestRegionServerMetrics {
     admin.deleteTable(tableName);
   }
 
-  public void waitTableDeleted(TableName name, long timeoutInMillis) throws Exception {
-    long start = System.currentTimeMillis();
-    while (true) {
-      HTableDescriptor[] tables = admin.listTables();
-      for (HTableDescriptor htd : tables) {
-        if (htd.getNameAsString() == name.getNameAsString())
-          return;
-      }
-      if (System.currentTimeMillis() - start > timeoutInMillis)
-        return;
-      Thread.sleep(1000);
-    }
-  }
-
   public void assertCounter(String metric, long expectedValue) {
     metricsHelper.assertCounter(metric, expectedValue, serverSource);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
index 04db81a..55da3f4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
@@ -198,7 +198,7 @@ public class TestRegionReplicaReplicationEndpoint {
 
     HTU.getAdmin().disableTable(htd.getTableName());
     htd.setRegionReplication(2);
-    HTU.getAdmin().modifyTable(htd.getTableName(), htd);
+    HTU.getAdmin().modifyTable(htd);
     HTU.getAdmin().enableTable(htd.getTableName());
 
     // assert peer configuration is correct
@@ -442,7 +442,7 @@ public class TestRegionReplicaReplicationEndpoint {
       HTU.getAdmin().deleteTable(toBeDisabledTable);
     } else if (disableReplication) {
       htd.setRegionReplication(regionReplication - 2);
-      HTU.getAdmin().modifyTable(toBeDisabledTable, htd);
+      HTU.getAdmin().modifyTable(htd);
       HTU.getAdmin().enableTable(toBeDisabledTable);
     }
     sinkWriter.append(toBeDisabledTable, encodedRegionName,
@@ -454,7 +454,7 @@ public class TestRegionReplicaReplicationEndpoint {
       // enable replication again so that we can verify replication
       HTU.getAdmin().disableTable(toBeDisabledTable); // disable the table
       htd.setRegionReplication(regionReplication);
-      HTU.getAdmin().modifyTable(toBeDisabledTable, htd);
+      HTU.getAdmin().modifyTable(htd);
       HTU.getAdmin().enableTable(toBeDisabledTable);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index cee34af..ad97790 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -1771,7 +1771,7 @@ public class TestAccessController extends SecureTestUtil {
 
       User newOwner = User.createUserForTesting(conf, "new_owner", new String[] {});
       htd.setOwner(newOwner);
-      admin.modifyTable(tableName, htd);
+      admin.modifyTable(htd);
 
       acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
       try {
@@ -2055,7 +2055,7 @@ public class TestAccessController extends SecureTestUtil {
   @Test
   public void testSnapshot() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
-    final HTableDescriptor htd = admin.getTableDescriptor(TEST_TABLE);
+    final HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(TEST_TABLE));
     final SnapshotDescription snapshot = new SnapshotDescription(
         TEST_TABLE.getNameAsString() + "-snapshot", TEST_TABLE);
     AccessTestAction snapshotAction = new AccessTestAction() {
@@ -2114,7 +2114,7 @@ public class TestAccessController extends SecureTestUtil {
   @Test
   public void testSnapshotWithOwner() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
-    final HTableDescriptor htd = admin.getTableDescriptor(TEST_TABLE);
+    final HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(TEST_TABLE));
     final SnapshotDescription snapshot = new SnapshotDescription(
         TEST_TABLE.getNameAsString() + "-snapshot", TEST_TABLE, null, USER_OWNER.getName());
 
@@ -2243,7 +2243,7 @@ public class TestAccessController extends SecureTestUtil {
         public Object run() throws Exception {
           try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
               Admin admin = conn.getAdmin()) {
-            return Arrays.asList(admin.listTables());
+            return admin.listTableDescriptors();
           }
         }
       };
@@ -2253,7 +2253,7 @@ public class TestAccessController extends SecureTestUtil {
         public Object run() throws Exception {
           try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
               Admin admin = conn.getAdmin()) {
-            return admin.getTableDescriptor(TEST_TABLE);
+            return admin.getDescriptor(TEST_TABLE);
           }
         }
       };
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
index f9b610a..d7a0277 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.Optional;
+import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Coprocessor;
@@ -115,7 +116,7 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
       Coprocessor.PRIORITY_USER, null);
     LOG.info("Modifying Table");
     try {
-      connection.getAdmin().modifyTable(TEST_TABLE, htd);
+      connection.getAdmin().modifyTable(htd);
       fail("Expected coprocessor to raise IOException");
     } catch (IOException e) {
       // swallow exception from coprocessor
@@ -160,8 +161,8 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
       new Path(coprocessorPath),
       Coprocessor.PRIORITY_USER, null);
     LOG.info("Modifying Table");
-    admin.modifyTable(TEST_TABLE, htd);
-    assertEquals(1, t.getTableDescriptor().getCoprocessors().size());
+    admin.modifyTable(htd);
+    assertEquals(1, t.getDescriptor().getCoprocessorDescriptors().size());
     LOG.info("Done Modifying Table");
   }
 
@@ -288,8 +289,8 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
     }
     LOG.info("Done Creating Table");
     // ensure table was not created
-    assertEquals(new HTableDescriptor[0],
-      admin.listTables("^" + TEST_TABLE.getNameAsString() + "$"));
+    assertEquals(0,
+      admin.listTableDescriptors(Pattern.compile("^" + TEST_TABLE.getNameAsString() + "$")).size());
   }
 
   public static class TestRegionObserver implements RegionCoprocessor, RegionObserver {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java
index 4fb5242..b95c776 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java
@@ -218,7 +218,7 @@ public class TestRpcAccessChecks {
 
   @Test
   public void testEnableCatalogJanitor() throws Exception {
-    verifyAdminCheckForAction((admin) -> admin.enableCatalogJanitor(true));
+    verifyAdminCheckForAction((admin) -> admin.catalogJanitorSwitch(true));
   }
 
   @Test
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
index 770bf3e..4829d5c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
@@ -683,7 +683,7 @@ public abstract class TestVisibilityLabels {
       HTableDescriptor htd = new HTableDescriptor(LABELS_TABLE_NAME);
       htd.addFamily(new HColumnDescriptor("f1"));
       htd.addFamily(new HColumnDescriptor("f2"));
-      admin.modifyTable(LABELS_TABLE_NAME, htd);
+      admin.modifyTable(htd);
       fail("Lables table should not get altered by user.");
     } catch (Exception e) {
     }
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 75dfc30..d39c0e6 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
@@ -777,7 +777,7 @@ public final class SnapshotTestingUtils {
     byte[][] splitKeys = getSplitKeys(nRegions);
     util.createTable(builder.build(), splitKeys);
     assertEquals((splitKeys.length + 1) * regionReplication,
-        util.getAdmin().getTableRegions(tableName).size());
+        util.getAdmin().getRegions(tableName).size());
   }
 
   public static byte[][] getSplitKeys() {
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 5a589d4..2f84c95 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
@@ -34,10 +34,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 import org.apache.hadoop.hbase.client.SnapshotType;
 import org.apache.hadoop.hbase.client.Table;
@@ -254,13 +254,13 @@ public class TestFlushSnapshotFromClient {
     // make sure the table doesn't exist
     boolean fail = false;
     do {
-    try {
-      admin.getTableDescriptor(tableName);
-      fail = true;
-      LOG.error("Table:" + tableName + " already exists, checking a new name");
-      tableName = TableName.valueOf(tableName+"!");
-    } catch (TableNotFoundException e) {
-      fail = false;
+      try {
+        admin.getDescriptor(tableName);
+        fail = true;
+        LOG.error("Table:" + tableName + " already exists, checking a new name");
+        tableName = TableName.valueOf(tableName + "!");
+      } catch (TableNotFoundException e) {
+        fail = false;
       }
     } while (fail);
 
@@ -280,7 +280,7 @@ public class TestFlushSnapshotFromClient {
         .setType(SnapshotProtos.SnapshotDescription.Type.FLUSH).build();
 
     // take the snapshot async
-    admin.takeSnapshotAsync(
+    admin.snapshotAsync(
       new SnapshotDescription("asyncSnapshot", TABLE_NAME, SnapshotType.FLUSH));
 
     // constantly loop, looking for the snapshot to complete
@@ -311,15 +311,15 @@ public class TestFlushSnapshotFromClient {
     SnapshotTestingUtils.waitForTableToBeOnline(UTIL, cloneBeforeMergeName);
 
     // Merge two regions
-    List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
-    Collections.sort(regions, new Comparator<HRegionInfo>() {
+    List<RegionInfo> regions = admin.getRegions(TABLE_NAME);
+    Collections.sort(regions, new Comparator<RegionInfo>() {
       @Override
-      public int compare(HRegionInfo r1, HRegionInfo r2) {
+      public int compare(RegionInfo r1, RegionInfo r2) {
         return Bytes.compareTo(r1.getStartKey(), r2.getStartKey());
       }
     });
 
-    int numRegions = admin.getTableRegions(TABLE_NAME).size();
+    int numRegions = admin.getRegions(TABLE_NAME).size();
     int numRegionsAfterMerge = numRegions - 2;
     admin.mergeRegionsAsync(regions.get(1).getEncodedNameAsBytes(),
         regions.get(2).getEncodedNameAsBytes(), true);
@@ -328,7 +328,7 @@ public class TestFlushSnapshotFromClient {
 
     // Verify that there's one region less
     waitRegionsAfterMerge(numRegionsAfterMerge);
-    assertEquals(numRegionsAfterMerge, admin.getTableRegions(TABLE_NAME).size());
+    assertEquals(numRegionsAfterMerge, admin.getRegions(TABLE_NAME).size());
 
     // Clone the table
     TableName cloneAfterMergeName = TableName.valueOf("cloneAfterMerge");
@@ -353,15 +353,15 @@ public class TestFlushSnapshotFromClient {
     SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, numRows, TEST_FAM);
 
     // Merge two regions
-    List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
-    Collections.sort(regions, new Comparator<HRegionInfo>() {
+    List<RegionInfo> regions = admin.getRegions(TABLE_NAME);
+    Collections.sort(regions, new Comparator<RegionInfo>() {
       @Override
-      public int compare(HRegionInfo r1, HRegionInfo r2) {
+      public int compare(RegionInfo r1, RegionInfo r2) {
         return Bytes.compareTo(r1.getStartKey(), r2.getStartKey());
       }
     });
 
-    int numRegions = admin.getTableRegions(TABLE_NAME).size();
+    int numRegions = admin.getRegions(TABLE_NAME).size();
     int numRegionsAfterMerge = numRegions - 2;
     admin.mergeRegionsAsync(regions.get(1).getEncodedNameAsBytes(),
         regions.get(2).getEncodedNameAsBytes(), true);
@@ -369,7 +369,7 @@ public class TestFlushSnapshotFromClient {
         regions.get(5).getEncodedNameAsBytes(), true);
 
     waitRegionsAfterMerge(numRegionsAfterMerge);
-    assertEquals(numRegionsAfterMerge, admin.getTableRegions(TABLE_NAME).size());
+    assertEquals(numRegionsAfterMerge, admin.getRegions(TABLE_NAME).size());
 
     // Take a snapshot
     String snapshotName = "snapshotAfterMerge";
@@ -436,7 +436,7 @@ public class TestFlushSnapshotFromClient {
         try {
           LOG.info("Submitting snapshot request: " + ClientSnapshotDescriptionUtils
               .toString(ProtobufUtil.createHBaseProtosSnapshotDesc(ss)));
-          admin.takeSnapshotAsync(ss);
+          admin.snapshotAsync(ss);
         } catch (Exception e) {
           LOG.info("Exception during snapshot request: " + ClientSnapshotDescriptionUtils.toString(
             ProtobufUtil.createHBaseProtosSnapshotDesc(ss))
@@ -514,7 +514,7 @@ public class TestFlushSnapshotFromClient {
       throws IOException, InterruptedException {
     // Verify that there's one region less
     long startTime = System.currentTimeMillis();
-    while (admin.getTableRegions(TABLE_NAME).size() != numRegionsAfterMerge) {
+    while (admin.getRegions(TABLE_NAME).size() != numRegionsAfterMerge) {
       // This may be flaky... if after 15sec the merge is not complete give up
       // it will fail in the assertEquals(numRegionsAfterMerge).
       if ((System.currentTimeMillis() - startTime) > 15000)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java
index cfe3b15..5ea8f3f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java
@@ -61,9 +61,9 @@ public class TestRestoreFlushSnapshotFromClient {
 
   protected final byte[] FAMILY = Bytes.toBytes("cf");
 
-  protected byte[] snapshotName0;
-  protected byte[] snapshotName1;
-  protected byte[] snapshotName2;
+  protected String snapshotName0;
+  protected String snapshotName1;
+  protected String snapshotName2;
   protected int snapshot0Rows;
   protected int snapshot1Rows;
   protected TableName tableName;
@@ -108,9 +108,9 @@ public class TestRestoreFlushSnapshotFromClient {
 
     long tid = System.currentTimeMillis();
     tableName = TableName.valueOf("testtb-" + tid);
-    snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
-    snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
-    snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
+    snapshotName0 = "snaptb0-" + tid;
+    snapshotName1 = "snaptb1-" + tid;
+    snapshotName2 = "snaptb2-" + tid;
 
     // create Table and disable it
     createTable();
@@ -121,7 +121,7 @@ public class TestRestoreFlushSnapshotFromClient {
     logFSTree();
 
     // take a snapshot
-    admin.snapshot(Bytes.toString(snapshotName0), tableName, SnapshotType.FLUSH);
+    admin.snapshot(snapshotName0, tableName, SnapshotType.FLUSH);
 
     LOG.info("=== after snapshot with 500 rows");
     logFSTree();
@@ -133,7 +133,7 @@ public class TestRestoreFlushSnapshotFromClient {
     logFSTree();
 
     // take a snapshot of the updated table
-    admin.snapshot(Bytes.toString(snapshotName1), tableName, SnapshotType.FLUSH);
+    admin.snapshot(snapshotName1, tableName, SnapshotType.FLUSH);
     LOG.info("=== after snapshot with 1000 rows");
     logFSTree();
     table.close();
@@ -184,7 +184,7 @@ public class TestRestoreFlushSnapshotFromClient {
     testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
   }
 
-  private void testCloneSnapshot(final TableName tableName, final byte[] snapshotName,
+  private void testCloneSnapshot(final TableName tableName, final String snapshotName,
       int snapshotRows) throws IOException, InterruptedException {
     // create a new table from snapshot
     admin.cloneSnapshot(snapshotName, tableName);
@@ -198,7 +198,7 @@ public class TestRestoreFlushSnapshotFromClient {
     TableName clonedTableName = TableName.valueOf("clonedtb-" + System.currentTimeMillis());
     admin.cloneSnapshot(snapshotName0, clonedTableName);
     verifyRowCount(UTIL, clonedTableName, snapshot0Rows);
-    admin.snapshot(Bytes.toString(snapshotName2), clonedTableName, SnapshotType.FLUSH);
+    admin.snapshot(snapshotName2, clonedTableName, SnapshotType.FLUSH);
     UTIL.deleteTable(clonedTableName);
 
     admin.cloneSnapshot(snapshotName2, clonedTableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
index 0855559..3686150 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
@@ -77,7 +77,7 @@ public class TestHBaseFsckMOB extends BaseTestHBaseFsck {
     connection = (ClusterConnection) TEST_UTIL.getConnection();
 
     admin = connection.getAdmin();
-    admin.setBalancerRunning(false, true);
+    admin.balancerSwitch(false, true);
 
     TEST_UTIL.waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
index f2973d6..5af2372 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -182,9 +183,9 @@ public class TestMiniClusterLoadSequential {
     return 1000;
   }
 
-  protected HColumnDescriptor getColumnDesc(Admin admin)
+  protected ColumnFamilyDescriptor getColumnDesc(Admin admin)
       throws TableNotFoundException, IOException {
-    return admin.getTableDescriptor(TABLE).getFamily(CF);
+    return admin.getDescriptor(TABLE).getColumnFamily(CF);
   }
 
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
index 531d08c..1329f44 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -47,6 +46,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -97,7 +97,7 @@ public class OfflineMetaRebuildTestCore {
     TEST_UTIL.startMiniCluster(3);
     conf = TEST_UTIL.getConfiguration();
     this.connection = ConnectionFactory.createConnection(conf);
-    assertEquals(0, TEST_UTIL.getAdmin().listTables().length);
+    assertEquals(0, TEST_UTIL.getAdmin().listTableDescriptors().size());
 
     // setup the table
     table = TableName.valueOf(TABLE_BASE + "-" + tableIdx);
@@ -109,7 +109,7 @@ public class OfflineMetaRebuildTestCore {
         + " entries.");
     assertEquals(16, tableRowCount(conf, table));
     TEST_UTIL.getAdmin().disableTable(table);
-    assertEquals(1, TEST_UTIL.getAdmin().listTables().length);
+    assertEquals(1, TEST_UTIL.getAdmin().listTableDescriptors().size());
   }
 
   @After
@@ -280,13 +280,11 @@ public class OfflineMetaRebuildTestCore {
     return MetaTableAccessor.fullScanRegions(TEST_UTIL.getConnection()).size();
   }
 
-  protected HTableDescriptor[] getTables(final Configuration configuration) throws IOException {
-    HTableDescriptor[] htbls = null;
+  protected List<TableDescriptor> getTables(final Configuration configuration) throws IOException {
     try (Connection connection = ConnectionFactory.createConnection(configuration)) {
       try (Admin admin = connection.getAdmin()) {
-        htbls = admin.listTables();
+        return admin.listTableDescriptors();
       }
     }
-    return htbls;
   }
 }
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 6b0e7c0..31545b5 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -144,8 +144,8 @@ module Hbase
       split_point_bytes = nil
       split_point_bytes = split_point.to_java_bytes unless split_point.nil?
       begin
-        @admin.splitRegion(table_or_region_name.to_java_bytes, split_point_bytes)
-      rescue java.lang.IllegalArgumentException => e
+        @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)
       end
     end
@@ -154,33 +154,26 @@ module Hbase
     # Enable/disable one split or merge switch
     # Returns previous switch setting.
     def splitormerge_switch(type, enabled)
-      switch_type = nil
       if type == 'SPLIT'
-        switch_type = org.apache.hadoop.hbase.client::MasterSwitchType::SPLIT
+        @admin.splitSwitch(java.lang.Boolean.valueOf(enabled), java.lang.Boolean.valueOf(false))
       elsif type == 'MERGE'
-        switch_type = org.apache.hadoop.hbase.client::MasterSwitchType::MERGE
+        @admin.mergeSwitch(java.lang.Boolean.valueOf(enabled), java.lang.Boolean.valueOf(false))
       else
         raise ArgumentError, 'only SPLIT or MERGE accepted for type!'
       end
-      @admin.setSplitOrMergeEnabled(
-        java.lang.Boolean.valueOf(enabled), java.lang.Boolean.valueOf(false),
-        switch_type
-      )[0]
     end
 
     #----------------------------------------------------------------------------------------------
     # Query the current state of the split or merge switch.
     # Returns the switch's state (true is enabled).
     def splitormerge_enabled(type)
-      switch_type = nil
       if type == 'SPLIT'
-        switch_type = org.apache.hadoop.hbase.client::MasterSwitchType::SPLIT
+        @admin.isSplitEnabled
       elsif type == 'MERGE'
-        switch_type = org.apache.hadoop.hbase.client::MasterSwitchType::MERGE
+        @admin.isMergeEnabled
       else
         raise ArgumentError, 'only SPLIT or MERGE accepted for type!'
       end
-      @admin.isSplitOrMergeEnabled(switch_type)
     end
 
     def locate_region(table_name, row_key)
@@ -203,7 +196,7 @@ module Hbase
     # Enable/disable balancer
     # Returns previous balancer switch setting.
     def balance_switch(enableDisable)
-      @admin.setBalancerRunning(
+      @admin.balancerSwitch(
         java.lang.Boolean.valueOf(enableDisable), java.lang.Boolean.valueOf(false)
       )
     end
@@ -232,7 +225,7 @@ module Hbase
     # Enable/disable region normalizer
     # Returns previous normalizer switch setting.
     def normalizer_switch(enableDisable)
-      @admin.setNormalizerRunning(java.lang.Boolean.valueOf(enableDisable))
+      @admin.normalizerSwitch(java.lang.Boolean.valueOf(enableDisable))
     end
 
     #----------------------------------------------------------------------------------------------
@@ -253,14 +246,14 @@ module Hbase
     # Request a scan of the catalog table (for garbage collection)
     # Returns an int signifying the number of entries cleaned
     def catalogjanitor_run
-      @admin.runCatalogScan
+      @admin.runCatalogJanitor
     end
 
     #----------------------------------------------------------------------------------------------
     # Enable/disable the catalog janitor
     # Returns previous catalog janitor switch setting.
     def catalogjanitor_switch(enableDisable)
-      @admin.enableCatalogJanitor(java.lang.Boolean.valueOf(enableDisable))
+      @admin.catalogJanitorSwitch(java.lang.Boolean.valueOf(enableDisable))
     end
 
     #----------------------------------------------------------------------------------------------
@@ -280,7 +273,7 @@ module Hbase
     # Enable/disable the cleaner chore
     # Returns previous cleaner switch setting.
     def cleaner_chore_switch(enableDisable)
-      @admin.setCleanerChoreRunning(java.lang.Boolean.valueOf(enableDisable))
+      @admin.cleanerChoreSwitch(java.lang.Boolean.valueOf(enableDisable))
     end
 
     #----------------------------------------------------------------------------------------------
@@ -301,8 +294,17 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Enables all tables matching the given regex
     def enable_all(regex)
-      regex = regex.to_s
-      @admin.enableTables(Pattern.compile(regex))
+      pattern = Pattern.compile(regex.to_s)
+      failed = java.util.ArrayList.new
+      admin.listTableNames(pattern).each do |table_name|
+        begin
+          admin.enableTable(table_name)
+        rescue java.io.IOException => e
+          puts "table:#{table_name}, error:#{e.toString}"
+          failed.add(table_name)
+        end
+      end
+      @failed
     end
 
     #----------------------------------------------------------------------------------------------
@@ -317,7 +319,16 @@ module Hbase
     # Disables all tables matching the given regex
     def disable_all(regex)
       pattern = Pattern.compile(regex.to_s)
-      @admin.disableTables(pattern).map { |t| t.getTableName.getNameAsString }
+      failed = java.util.ArrayList.new
+      admin.listTableNames(pattern).each do |table_name|
+        begin
+          admin.disableTable(table_name)
+        rescue java.io.IOException => e
+          puts "table:#{table_name}, error:#{e.toString}"
+          failed.add(table_name)
+        end
+      end
+      @failed
     end
 
     #---------------------------------------------------------------------------------------------
@@ -347,8 +358,16 @@ module Hbase
     # Drops a table
     def drop_all(regex)
       pattern = Pattern.compile(regex.to_s)
-      failed = @admin.deleteTables(pattern).map { |t| t.getTableName.getNameAsString }
-      failed
+      failed = java.util.ArrayList.new
+      admin.listTableNames(pattern).each do |table_name|
+        begin
+          admin.deleteTable(table_name)
+        rescue java.io.IOException => e
+          puts puts "table:#{table_name}, error:#{e.toString}"
+          failed.add(table_name)
+        end
+      end
+      @failed
     end
 
     #----------------------------------------------------------------------------------------------
@@ -492,17 +511,17 @@ module Hbase
     # Returns table's structure description
     def describe(table_name)
       tableExists(table_name)
-      @admin.getTableDescriptor(TableName.valueOf(table_name)).to_s
+      @admin.getDescriptor(TableName.valueOf(table_name)).to_s
     end
 
     def get_column_families(table_name)
       tableExists(table_name)
-      @admin.getTableDescriptor(TableName.valueOf(table_name)).getColumnFamilies
+      @admin.getDescriptor(TableName.valueOf(table_name)).getColumnFamilies
     end
 
     def get_table_attributes(table_name)
       tableExists(table_name)
-      @admin.getTableDescriptor(TableName.valueOf(table_name)).toStringTableAttributes
+      @admin.getDescriptor(TableName.valueOf(table_name)).toStringTableAttributes
     end
 
     #----------------------------------------------------------------------------------------------
@@ -510,7 +529,7 @@ module Hbase
     def truncate(table_name_str)
       puts "Truncating '#{table_name_str}' table (it may take a while):"
       table_name = TableName.valueOf(table_name_str)
-      table_description = @admin.getTableDescriptor(table_name)
+      table_description = @admin.getDescriptor(table_name)
       raise ArgumentError, "Table #{table_name_str} is not enabled. Enable it first." unless
           enabled?(table_name_str)
       puts 'Disabling table...'
@@ -551,7 +570,7 @@ module Hbase
         locator.close
       end
 
-      table_description = @admin.getTableDescriptor(table_name)
+      table_description = @admin.getDescriptor(table_name)
       puts 'Disabling table...'
       disable(table_name_str)
 
@@ -627,7 +646,7 @@ module Hbase
       table_name = TableName.valueOf(table_name_str)
 
       # Get table descriptor
-      htd = org.apache.hadoop.hbase.HTableDescriptor.new(@admin.getTableDescriptor(table_name))
+      htd = org.apache.hadoop.hbase.HTableDescriptor.new(@admin.getDescriptor(table_name))
       hasTableUpdate = false
 
       # Process all args
@@ -742,17 +761,17 @@ module Hbase
 
       # Bulk apply all table modifications.
       if hasTableUpdate
-        @admin.modifyTable(table_name, htd)
+        future = @admin.modifyTableAsync(htd)
 
         if wait == true
           puts 'Updating all regions with the new schema...'
-          alter_status(table_name_str)
+          future.get
         end
       end
     end
 
     def status(format, type)
-      status = @admin.getClusterStatus
+      status = org.apache.hadoop.hbase.ClusterStatus.new(@admin.getClusterMetrics)
       if format == 'detailed'
         puts(format('version %s', status.getHBaseVersion))
         # Put regions in transition first because usually empty
@@ -767,7 +786,7 @@ module Hbase
           puts(format('    %s:%d %d', server.getHostname, server.getPort, server.getStartcode))
         end
 
-        master_coprocs = java.util.Arrays.toString(@admin.getMasterCoprocessors)
+        master_coprocs = @admin.getMasterCoprocessorNames.toString
         unless master_coprocs.nil?
           puts(format('master coprocessors: %s', master_coprocs))
         end
@@ -1124,13 +1143,13 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Returns the ClusterStatus of the cluster
     def getClusterStatus
-      @admin.getClusterStatus
+      org.apache.hadoop.hbase.ClusterStatus.new(@admin.getClusterMetrics)
     end
 
     #----------------------------------------------------------------------------------------------
     # Returns a list of regionservers
     def getRegionServers
-      @admin.getClusterStatus.getServers.map { |serverName| serverName }
+      org.apache.hadoop.hbase.ClusterStatus.new(@admin.getClusterMetrics).getServers.map { |serverName| serverName }
     end
 
     #----------------------------------------------------------------------------------------------
@@ -1393,7 +1412,7 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # List live region servers
     def list_liveservers
-      @admin.getClusterStatus.getServers.to_a
+      org.apache.hadoop.hbase.ClusterStatus.new(@admin.getClusterMetrics).getServers.to_a
     end
 
     #---------------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/quotas.rb b/hbase-shell/src/main/ruby/hbase/quotas.rb
index f67ee75..d4d73e9 100644
--- a/hbase-shell/src/main/ruby/hbase/quotas.rb
+++ b/hbase-shell/src/main/ruby/hbase/quotas.rb
@@ -238,30 +238,25 @@ module Hbase
       raise(ArgumentError, 'Unexpected arguments: ' + args.inspect) unless args.empty?
 
       # Start the scanner
-      scanner = @admin.getQuotaRetriever(filter)
-      begin
-        iter = scanner.iterator
+      quotas = @admin.getQuota(filter)
+      iter = quotas.iterator
 
-        # Iterate results
-        while iter.hasNext
-          break if limit > 0 && count >= limit
+      # Iterate results
+      while iter.hasNext
+        break if limit > 0 && count >= limit
 
-          settings = iter.next
-          owner = {
-            USER => settings.getUserName,
-            TABLE => settings.getTableName,
-            NAMESPACE => settings.getNamespace,
-            REGIONSERVER => settings.getRegionServer
-          }.delete_if { |_k, v| v.nil? }.map { |k, v| k.to_s + ' => ' + v.to_s } * ', '
+        settings = iter.next
+        owner = {
+          USER => settings.getUserName,
+          TABLE => settings.getTableName,
+          NAMESPACE => settings.getNamespace,
+          REGIONSERVER => settings.getRegionServer
+        }.delete_if { |_k, v| v.nil? }.map { |k, v| k.to_s + ' => ' + v.to_s } * ', '
 
-          yield owner, settings.to_s
+        yield owner, settings.to_s
 
-          count += 1
-        end
-      ensure
-        scanner.close
+        count += 1
       end
-
       count
     end
 
diff --git a/hbase-shell/src/main/ruby/hbase/security.rb b/hbase-shell/src/main/ruby/hbase/security.rb
index 5275898..4639c71 100644
--- a/hbase-shell/src/main/ruby/hbase/security.rb
+++ b/hbase-shell/src/main/ruby/hbase/security.rb
@@ -66,7 +66,7 @@ module Hbase
             raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
 
             tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name)
-            htd = @admin.getTableDescriptor(tableName)
+            htd = org.apache.hadoop.hbase.HTableDescriptor.new(@admin.getDescriptor(tableName))
 
             unless family.nil?
               raise(ArgumentError, "Can't find a family: #{family}") unless htd.hasFamily(family.to_java_bytes)
@@ -111,7 +111,7 @@ module Hbase
             raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
 
             tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name)
-            htd = @admin.getTableDescriptor(tableName)
+            htd = org.apache.hadoop.hbase.HTableDescriptor.new(@admin.getDescriptor(tableName))
 
             unless family.nil?
               raise(ArgumentError, "Can't find a family: #{family}") unless htd.hasFamily(family.to_java_bytes)
diff --git a/hbase-shell/src/main/ruby/hbase/taskmonitor.rb b/hbase-shell/src/main/ruby/hbase/taskmonitor.rb
index 76f8711..ca4180a 100644
--- a/hbase-shell/src/main/ruby/hbase/taskmonitor.rb
+++ b/hbase-shell/src/main/ruby/hbase/taskmonitor.rb
@@ -79,7 +79,7 @@ module Hbase
       java_import 'java.io.InputStreamReader'
       java_import 'org.apache.hbase.thirdparty.com.google.gson.JsonParser'
 
-      infoport = @admin.getClusterStatus.getLoad(host).getInfoServerPort.to_s
+      infoport = org.apache.hadoop.hbase.ClusterStatus.new(@admin.getClusterMetrics).getLoad(host).getInfoServerPort.to_s
 
       # Note: This condition use constants from hbase-server
       # if (!@admin.getConfiguration().getBoolean(org.apache.hadoop.hbase.http.ServerConfigurationKeys::HBASE_SSL_ENABLED_KEY,
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 a112cef..53fbfff 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
@@ -54,7 +54,7 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Pattern;
-
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -705,13 +705,7 @@ public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements TH
   @Override
   public boolean isTableAvailableWithSplit(TTableName tableName, List<ByteBuffer> splitKeys)
       throws TIOError, TException {
-    try {
-      TableName table = tableNameFromThrift(tableName);
-      byte[][] split = splitKeyFromThrift(splitKeys);
-      return connectionCache.getAdmin().isTableAvailable(table, split);
-    } catch (IOException e) {
-      throw getTIOError(e);
-    }
+    throw new NotImplementedException("isTableAvailableWithSplit not supported");
   }
 
   @Override
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 888256e..aa0c2a8 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
@@ -31,8 +31,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.RegionMetrics;
@@ -53,7 +51,6 @@ 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.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -68,7 +65,6 @@ import org.apache.hadoop.hbase.thrift2.generated.TNamespaceDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.TTableName;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransport;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -112,7 +108,7 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void close() throws IOException {
+  public void close() {
     transport.close();
   }
 
@@ -137,40 +133,16 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] listTables() throws IOException {
-    return listTables((String)null);
-  }
-
-  @Override
   public List<TableDescriptor> listTableDescriptors() throws IOException {
     return listTableDescriptors((Pattern) null);
   }
 
   @Override
-  public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
-    String regex = (pattern == null ? null : pattern.toString());
-    return listTables(regex);
-  }
-
-  @Override
   public List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException {
     return listTableDescriptors(pattern, false);
   }
 
   @Override
-  public HTableDescriptor[] listTables(String regex) throws IOException {
-    return listTables(regex, false);
-  }
-
-  @Override
-  public HTableDescriptor[] listTables(Pattern pattern, boolean includeSysTables)
-      throws IOException {
-    String regex = (pattern == null ? null : pattern.toString());
-    return listTables(regex, includeSysTables);
-
-  }
-
-  @Override
   public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
       throws IOException {
     try {
@@ -185,20 +157,8 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] listTables(String regex, boolean includeSysTables) throws IOException {
-    try {
-      List<TTableDescriptor> tTableDescriptors = client
-          .getTableDescriptorsByPattern(regex, includeSysTables);
-      return ThriftUtilities.hTableDescriptorsFromThrift(tTableDescriptors);
-
-    } catch (TException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
   public TableName[] listTableNames() throws IOException {
-    return listTableNames((String)null);
+    return listTableNames(null);
   }
 
   @Override
@@ -207,18 +167,8 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public TableName[] listTableNames(String regex) throws IOException {
-    return listTableNames(regex, false);
-  }
-
-  @Override
   public TableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException {
     String regex = (pattern == null ? null : pattern.toString());
-    return listTableNames(regex, includeSysTables);
-  }
-
-  @Override
-  public TableName[] listTableNames(String regex, boolean includeSysTables) throws IOException {
     try {
       List<TTableName> tTableNames = client.getTableNamesByPattern(regex, includeSysTables);
       return ThriftUtilities.tableNamesArrayFromThrift(tTableNames);
@@ -228,18 +178,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor getTableDescriptor(TableName tableName)
-      throws TableNotFoundException, IOException {
-    TTableName tTableName = ThriftUtilities.tableNameFromHBase(tableName);
-    try {
-      TTableDescriptor tTableDescriptor = client.getTableDescriptor(tTableName);
-      return ThriftUtilities.hTableDescriptorFromThrift(tTableDescriptor);
-    } catch (TException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
   public TableDescriptor getDescriptor(TableName tableName)
       throws TableNotFoundException, IOException {
     TTableName tTableName = ThriftUtilities.tableNameFromHBase(tableName);
@@ -252,16 +190,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] listTableDescriptorsByNamespace(String name) throws IOException {
-    try {
-      List<TTableDescriptor> tTableDescriptors = client.getTableDescriptorsByNamespace(name);
-      return ThriftUtilities.hTableDescriptorsFromThrift(tTableDescriptors);
-    } catch (TException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
   public List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException {
     try {
       List<TTableDescriptor> tTableDescriptors = client
@@ -388,17 +316,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException {
-    TTableName tTableName = ThriftUtilities.tableNameFromHBase(tableName);
-    List<ByteBuffer> splitKeyInBuffer = ThriftUtilities.splitKeyFromHBase(splitKeys);
-    try {
-      return client.isTableAvailableWithSplit(tTableName, splitKeyInBuffer);
-    } catch (TException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
   public void addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
       throws IOException {
     TTableName tTableName = ThriftUtilities.tableNameFromHBase(tableName);
@@ -412,11 +329,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void deleteColumn(TableName tableName, byte[] columnFamily) throws IOException {
-    deleteColumnFamily(tableName, columnFamily);
-  }
-
-  @Override
   public void deleteColumnFamily(TableName tableName, byte[] columnFamily) throws IOException {
     TTableName tTableName = ThriftUtilities.tableNameFromHBase(tableName);
     try {
@@ -440,11 +352,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void modifyTable(TableName tableName, TableDescriptor td) throws IOException {
-    modifyTable(td);
-  }
-
-  @Override
   public void modifyTable(TableDescriptor td) throws IOException {
     TTableDescriptor tTableDescriptor = ThriftUtilities
         .tableDescriptorFromHBase(td);
@@ -525,85 +432,12 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] disableTables(String regex) throws IOException {
-    throw new NotImplementedException("disableTables by pattern not supported in ThriftAdmin");
-  }
-
-  @Override
-  public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
-    throw new NotImplementedException("disableTables by pattern not supported in ThriftAdmin");
-  }
-
-  @Override
-  public HTableDescriptor[] enableTables(String regex) throws IOException {
-    throw new NotImplementedException("enableTables by pattern not supported in ThriftAdmin");
-  }
-
-  @Override
-  public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
-    throw new NotImplementedException("enableTables by pattern not supported in ThriftAdmin");
-  }
-
-  @Override
-  public HTableDescriptor[] deleteTables(String regex) throws IOException {
-    throw new NotImplementedException("deleteTables by pattern not supported in ThriftAdmin");
-  }
-
-  @Override
-  public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
-    throw new NotImplementedException("deleteTables by pattern not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames)
-      throws IOException {
-    throw new NotImplementedException("getTableDescriptorsByTableName not supported in ThriftAdmin"
-        + ", use getDescriptor to get descriptors one by one");
-  }
-
-  @Override
   public List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException {
     throw new NotImplementedException("listTableDescriptors not supported in ThriftAdmin"
         + ", use getDescriptor to get descriptors one by one");
   }
 
   @Override
-  public HTableDescriptor[] getTableDescriptors(List<String> names) throws IOException {
-    throw new NotImplementedException("getTableDescriptors not supported in ThriftAdmin"
-        + ", use getDescriptor to get descriptors one by one");
-  }
-
-  @Override
-  public void closeRegion(String regionname, String serverName) {
-    throw new NotImplementedException("closeRegion not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public void closeRegion(byte[] regionname, String serverName) {
-    throw new NotImplementedException("closeRegion not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public boolean closeRegionWithEncodedRegionName(String encodedRegionName, String serverName) {
-    throw new NotImplementedException(
-        "closeRegionWithEncodedRegionName not supported in ThriftAdmin");
-  }
-
-  @Override
-  public void closeRegion(ServerName sn, HRegionInfo hri) {
-    throw new NotImplementedException("closeRegion not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public List<HRegionInfo> getOnlineRegions(ServerName sn) {
-    throw new NotImplementedException("getOnlineRegions not supported in ThriftAdmin");
-  }
-
-  @Override
   public List<RegionInfo> getRegions(ServerName serverName) {
     throw new NotImplementedException("getRegions not supported in ThriftAdmin");
   }
@@ -811,12 +645,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) {
-    throw new NotImplementedException("mergeRegions not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public Future<Void> mergeRegionsAsync(byte[] nameOfRegionA, byte[] nameOfRegionB,
       boolean forcible) {
     throw new NotImplementedException("mergeRegionsAsync not supported in ThriftAdmin");
@@ -833,31 +661,16 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void splitRegion(byte[] regionName) {
-    throw new NotImplementedException("splitRegion not supported in ThriftAdmin");
-  }
-
-  @Override
   public void split(TableName tableName, byte[] splitPoint) {
     throw new NotImplementedException("split not supported in ThriftAdmin");
   }
 
   @Override
-  public void splitRegion(byte[] regionName, byte[] splitPoint) {
-    throw new NotImplementedException("splitRegion not supported in ThriftAdmin");
-  }
-
-  @Override
   public Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) {
     throw new NotImplementedException("splitRegionAsync not supported in ThriftAdmin");
   }
 
   @Override
-  public Future<Void> modifyTableAsync(TableName tableName, TableDescriptor td) {
-    throw new NotImplementedException("modifyTableAsync not supported in ThriftAdmin");
-  }
-
-  @Override
   public Future<Void> modifyTableAsync(TableDescriptor td) {
     throw new NotImplementedException("modifyTableAsync not supported in ThriftAdmin");
   }
@@ -906,11 +719,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public List<HRegionInfo> getTableRegions(TableName tableName) {
-    throw new NotImplementedException("getTableRegions not supported in ThriftAdmin");
-  }
-
-  @Override
   public List<RegionInfo> getRegions(TableName tableName) {
     throw new NotImplementedException("getRegions not supported in ThriftAdmin");
   }
@@ -975,12 +783,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void snapshot(byte[] snapshotName, TableName tableName) {
-    throw new NotImplementedException("snapshot not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public void snapshot(String snapshotName, TableName tableName, SnapshotType type) {
     throw new NotImplementedException("snapshot not supported in ThriftAdmin");
 
@@ -1004,22 +806,10 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void restoreSnapshot(byte[] snapshotName) {
-    throw new NotImplementedException("restoreSnapshot not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public void restoreSnapshot(String snapshotName) {
     throw new NotImplementedException("restoreSnapshot not supported in ThriftAdmin");
 
   }
-
-  @Override
-  public Future<Void> restoreSnapshotAsync(String snapshotName) {
-    throw new NotImplementedException("restoreSnapshotAsync not supported in ThriftAdmin");
-  }
-
   @Override
   public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot,
       boolean restoreAcl) {
@@ -1053,24 +843,12 @@ public class ThriftAdmin implements Admin {
   public List<SnapshotDescription> listSnapshots() {
     throw new NotImplementedException("listSnapshots not supported in ThriftAdmin");
   }
-
-  @Override
-  public List<SnapshotDescription> listSnapshots(String regex) {
-    throw new NotImplementedException("listSnapshots not supported in ThriftAdmin");
-  }
-
   @Override
   public List<SnapshotDescription> listSnapshots(Pattern pattern) {
     throw new NotImplementedException("listSnapshots not supported in ThriftAdmin");
   }
 
   @Override
-  public List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
-      String snapshotNameRegex) {
-    throw new NotImplementedException("listTableSnapshots not supported in ThriftAdmin");
-  }
-
-  @Override
   public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
       Pattern snapshotNamePattern) {
     throw new NotImplementedException("listTableSnapshots not supported in ThriftAdmin");
@@ -1087,21 +865,11 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void deleteSnapshots(String regex) {
-    throw new NotImplementedException("deleteSnapshots not supported in ThriftAdmin");
-  }
-
-  @Override
   public void deleteSnapshots(Pattern pattern) {
     throw new NotImplementedException("deleteSnapshots not supported in ThriftAdmin");
   }
 
   @Override
-  public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) {
-    throw new NotImplementedException("deleteTableSnapshots not supported in ThriftAdmin");
-  }
-
-  @Override
   public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) {
     throw new NotImplementedException("deleteTableSnapshots not supported in ThriftAdmin");
   }
@@ -1112,11 +880,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public QuotaRetriever getQuotaRetriever(QuotaFilter filter) {
-    throw new NotImplementedException("getQuotaRetriever not supported in ThriftAdmin");
-  }
-
-  @Override
   public List<QuotaSettings> getQuota(QuotaFilter filter) {
     throw new NotImplementedException("getQuota not supported in ThriftAdmin");
   }
@@ -1294,16 +1057,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public Pair<Integer, Integer> getAlterStatus(TableName tableName) {
-    throw new NotImplementedException("getAlterStatus not supported in ThriftAdmin");
-  }
-
-  @Override
-  public Pair<Integer, Integer> getAlterStatus(byte[] tableName) {
-    throw new NotImplementedException("getAlterStatus not supported in ThriftAdmin");
-  }
-
-  @Override
   public Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily) {
     throw new NotImplementedException("deleteColumnFamilyAsync not supported in ThriftAdmin");
   }