You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2017/11/17 01:49:47 UTC

hbase git commit: HBASE-18911 Unify Admin and AsyncAdmin's methods name

Repository: hbase
Updated Branches:
  refs/heads/master d72649283 -> 52273aa8f


HBASE-18911 Unify Admin and AsyncAdmin's methods name


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/52273aa8
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/52273aa8
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/52273aa8

Branch: refs/heads/master
Commit: 52273aa8f3221e11489004bacba4f4b6eb05f5c3
Parents: d726492
Author: Guanghao Zhang <zg...@apache.org>
Authored: Sat Oct 14 17:41:15 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Fri Nov 17 09:46:54 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   | 185 ++++++++++++++++---
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  66 ++++---
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |  82 ++++----
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  97 ++++++----
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  76 +++++---
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  15 ++
 .../hadoop/hbase/client/TestInterfaceAlign.java |  91 +++++++++
 .../hbase/client/TestAsyncClusterAdminApi.java  |   6 +-
 .../client/TestAsyncDecommissionAdminApi.java   |   2 +-
 .../client/TestAsyncProcedureAdminApi.java      |   2 +-
 .../hbase/client/TestAsyncRegionAdminApi.java   |  38 ++--
 ...estAsyncReplicationAdminApiWithClusters.java |   8 +-
 .../hbase/client/TestAsyncTableAdminApi.java    |  16 +-
 .../hbase/client/TestAsyncToolAdminApi.java     |  40 ++--
 14 files changed, 520 insertions(+), 204 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
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 d22d9e1..fd02a48 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
@@ -26,6 +26,7 @@ import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.concurrent.Future;
 import java.util.regex.Pattern;
 
@@ -58,6 +59,7 @@ 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;
 
 /**
@@ -916,15 +918,43 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * 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).
+   * 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
+   * @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)}.
    */
-  void compactRegionServer(ServerName sn, boolean major)
-    throws IOException, InterruptedException;
+  @Deprecated
+  default void compactRegionServer(ServerName sn, boolean major) throws IOException,
+      InterruptedException {
+    if (major) {
+      majorCompactRegionServer(sn);
+    } else {
+      compactRegionServer(sn);
+    }
+  }
+
+  /**
+   * 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 serverName the region server name
+   * @throws IOException if a remote or network exception occurs
+   */
+  void compactRegionServer(ServerName serverName) throws IOException;
+
+  /**
+   * Major 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 serverName the region server name
+   * @throws IOException if a remote or network exception occurs
+   */
+  void majorCompactRegionServer(ServerName serverName) throws IOException;
 
   /**
    * Move the region <code>r</code> to <code>dest</code>.
@@ -1381,13 +1411,74 @@ public interface Admin extends Abortable, Closeable {
   ClusterStatus getClusterStatus(EnumSet<Option> options) throws IOException;
 
   /**
+   * @return current master server name
+   * @throws IOException if a remote or network exception occurs
+   */
+  default ServerName getMaster() throws IOException {
+    return getClusterStatus(EnumSet.of(Option.MASTER)).getMaster();
+  }
+
+  /**
+   * @return current backup master list
+   * @throws IOException if a remote or network exception occurs
+   */
+  default Collection<ServerName> getBackupMasters() throws IOException {
+    return getClusterStatus(EnumSet.of(Option.BACKUP_MASTERS)).getBackupMasters();
+  }
+
+  /**
+   * @return current live region servers list
+   * @throws IOException if a remote or network exception occurs
+   */
+  default Collection<ServerName> getRegionServers() throws IOException {
+    return getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers();
+  }
+
+  /**
    * Get {@link RegionLoad} of all regions hosted on a regionserver.
    *
    * @param serverName region server from which regionload is required.
    * @return region load map of all regions hosted on a region server
    * @throws IOException if a remote or network exception occurs
+   * @deprecated since 2.0 version and will be removed in 3.0 version.
+   *             use {@link #getRegionLoads(ServerName)}
+   */
+  @Deprecated
+  default Map<byte[], RegionLoad> getRegionLoad(ServerName serverName) throws IOException {
+    return getRegionLoad(serverName, null);
+  }
+
+  /**
+   * Get {@link RegionLoad} of all regions hosted on a regionserver.
+   *
+   * @param serverName region server from which regionload is required.
+   * @return a region load list of all regions hosted on a region server
+   * @throws IOException if a remote or network exception occurs
+   */
+  default List<RegionLoad> getRegionLoads(ServerName serverName) throws IOException {
+    return getRegionLoads(serverName, null);
+  }
+
+  /**
+   * Get {@link RegionLoad} of all regions hosted on a regionserver for a table.
+   *
+   * @param serverName region server from which regionload is required.
+   * @param tableName get region load of regions belonging to the table
+   * @return region load map of all regions of a table hosted on a region server
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated since 2.0 version and will be removed in 3.0 version.
+   *             use {@link #getRegionLoads(ServerName, TableName)}
    */
-  Map<byte[], RegionLoad> getRegionLoad(ServerName serverName) throws IOException;
+  @Deprecated
+  default Map<byte[], RegionLoad> getRegionLoad(ServerName serverName, TableName tableName)
+      throws IOException {
+    List<RegionLoad> regionLoads = getRegionLoads(serverName, tableName);
+    Map<byte[], RegionLoad> resultMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    for (RegionLoad regionLoad : regionLoads) {
+      resultMap.put(regionLoad.getName(), regionLoad);
+    }
+    return resultMap;
+  }
 
   /**
    * Get {@link RegionLoad} of all regions hosted on a regionserver for a table.
@@ -1397,7 +1488,7 @@ public interface Admin extends Abortable, Closeable {
    * @return region load map of all regions of a table hosted on a region server
    * @throws IOException if a remote or network exception occurs
    */
-  Map<byte[], RegionLoad> getRegionLoad(ServerName serverName, TableName tableName) throws IOException;
+  List<RegionLoad> getRegionLoads(ServerName serverName, TableName tableName) throws IOException;
 
   /**
    * @return Configuration used by the instance.
@@ -2135,14 +2226,23 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * 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
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException;
+
+  /**
    * Creates and returns a {@link com.google.protobuf.RpcChannel} instance connected to the active
    * master. <p> The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access
    * a published coprocessor {@link com.google.protobuf.Service} using standard protobuf service
@@ -2285,49 +2385,82 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * 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 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 #splitOrMergeEnabledSwitch(boolean, boolean, MasterSwitchType...)}.
+   * @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 {
-    return splitOrMergeEnabledSwitch(enabled, synchronous, switchTypes);
+      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 or Merge switches on or off.
-   *
+   * 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, to return.
-   * @param switchTypes switchType list {@link MasterSwitchType}
-   * @return Previous switch value array
+   * @param synchronous If <code>true</code>, it waits until current split() call, if outstanding,
+   *          to return.
+   * @return Previous switch value
    */
-  boolean[] splitOrMergeEnabledSwitch(boolean enabled, boolean synchronous,
-                                   MasterSwitchType... switchTypes) throws IOException;
+  boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException;
+
+  /**
+   * Turn the merge switch on or off.
+   * @param enabled enabled or not
+   * @param synchronous If <code>true</code>, it waits until current merge() call, if outstanding,
+   *          to return.
+   * @return Previous switch value
+   */
+  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 #splitOrMergeEnabledSwitch(MasterSwitchType)}} instead.
+   * {@link #isSplitEnabled()} or {@link #isMergeEnabled()} instead.
    */
   @Deprecated
   default boolean isSplitOrMergeEnabled(MasterSwitchType switchType) throws IOException {
-    return splitOrMergeEnabledSwitch(switchType);
+    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 switch.
-   *
+   * Query the current state of the split switch.
+   * @return <code>true</code> if the switch is enabled, <code>false</code> otherwise.
+   */
+  boolean isSplitEnabled() throws IOException;
+
+  /**
+   * Query the current state of the merge switch.
    * @return <code>true</code> if the switch is enabled, <code>false</code> otherwise.
    */
-  boolean splitOrMergeEnabledSwitch(MasterSwitchType switchType) throws IOException;
+  boolean isMergeEnabled() throws IOException;
 
   /**
    * Add a new replication peer for replicating data to slave cluster.

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index c716441..0e0a673 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import com.google.protobuf.RpcChannel;
 
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
@@ -61,8 +62,8 @@ public interface AsyncAdmin {
    * List all the userspace tables.
    * @return - returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
    */
-  default CompletableFuture<List<TableDescriptor>> listTables() {
-    return listTables(false);
+  default CompletableFuture<List<TableDescriptor>> listTableDescriptors() {
+    return listTableDescriptors(false);
   }
 
   /**
@@ -70,7 +71,7 @@ public interface AsyncAdmin {
    * @param includeSysTables False to match only against userspace tables
    * @return - returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<List<TableDescriptor>> listTables(boolean includeSysTables);
+  CompletableFuture<List<TableDescriptor>> listTableDescriptors(boolean includeSysTables);
 
   /**
    * List all the tables matching the given pattern.
@@ -78,7 +79,15 @@ public interface AsyncAdmin {
    * @param includeSysTables False to match only against userspace tables
    * @return - returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<List<TableDescriptor>> listTables(Pattern pattern, boolean includeSysTables);
+  CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern,
+      boolean includeSysTables);
+
+  /**
+   * Get list of table descriptors by namespace.
+   * @param name namespace name
+   * @return returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
+   */
+  CompletableFuture<List<TableDescriptor>> listTableDescriptorsByNamespace(String name);
 
   /**
    * List all of the names of userspace tables.
@@ -105,11 +114,18 @@ public interface AsyncAdmin {
   CompletableFuture<List<TableName>> listTableNames(Pattern pattern, boolean includeSysTables);
 
   /**
+   * Get list of table names by namespace.
+   * @param name namespace name
+   * @return The list of table names in the namespace wrapped by a {@link CompletableFuture}.
+   */
+  CompletableFuture<List<TableName>> listTableNamesByNamespace(String name);
+
+  /**
    * Method for getting the tableDescriptor
    * @param tableName as a {@link TableName}
    * @return the read-only tableDescriptor wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<TableDescriptor> getTableDescriptor(TableName tableName);
+  CompletableFuture<TableDescriptor> getDescriptor(TableName tableName);
 
   /**
    * Creates a new table.
@@ -140,7 +156,7 @@ public interface AsyncAdmin {
    */
   CompletableFuture<Void> createTable(TableDescriptor desc, byte[][] splitKeys);
 
-  /*
+  /**
    * Modify an existing table, more IRB friendly version.
    * @param desc modified description of the table
    */
@@ -259,12 +275,12 @@ public interface AsyncAdmin {
   /**
    * Get all the online regions on a region server.
    */
-  CompletableFuture<List<RegionInfo>> getOnlineRegions(ServerName serverName);
+  CompletableFuture<List<RegionInfo>> getRegions(ServerName serverName);
 
   /**
    * Get the regions of a given table.
    */
-  CompletableFuture<List<RegionInfo>> getTableRegions(TableName tableName);
+  CompletableFuture<List<RegionInfo>> getRegions(TableName tableName);
 
   /**
    * Flush a table.
@@ -363,28 +379,28 @@ public interface AsyncAdmin {
    * @param on
    * @return Previous switch value wrapped by a {@link CompletableFuture}
    */
-  CompletableFuture<Boolean> setMergeOn(boolean on);
+  CompletableFuture<Boolean> mergeSwitch(boolean on);
 
   /**
    * Query the current state of the Merge switch.
    * @return true if the switch is on, false otherwise. The return value will be wrapped by a
    *         {@link CompletableFuture}
    */
-  CompletableFuture<Boolean> isMergeOn();
+  CompletableFuture<Boolean> isMergeEnabled();
 
   /**
    * Turn the Split switch on or off.
    * @param on
    * @return Previous switch value wrapped by a {@link CompletableFuture}
    */
-  CompletableFuture<Boolean> setSplitOn(boolean on);
+  CompletableFuture<Boolean> splitSwitch(boolean on);
 
   /**
    * Query the current state of the Split switch.
    * @return true if the switch is on, false otherwise. The return value will be wrapped by a
    *         {@link CompletableFuture}
    */
-  CompletableFuture<Boolean> isSplitOn();
+  CompletableFuture<Boolean> isSplitEnabled();
 
   /**
    * Merge two regions.
@@ -746,7 +762,7 @@ public interface AsyncAdmin {
    * @param props Property/Value pairs of properties passing to the procedure
    * @return data returned after procedure execution. null if no return data.
    */
-  CompletableFuture<byte[]> execProcedureWithRet(String signature, String instance,
+  CompletableFuture<byte[]> execProcedureWithReturn(String signature, String instance,
       Map<String, String> props);
 
   /**
@@ -843,6 +859,14 @@ public interface AsyncAdmin {
   }
 
   /**
+   * @return a list of master coprocessors wrapped by {@link CompletableFuture}
+   */
+  default CompletableFuture<List<String>> getMasterCoprocessors() {
+    return getClusterStatus(EnumSet.of(Option.MASTER_COPROCESSORS))
+        .thenApply(ClusterStatus::getMasterCoprocessors).thenApply(Arrays::asList);
+  }
+
+  /**
    * Get the info port of the current master if one is available.
    * @return master info port
    */
@@ -966,7 +990,7 @@ public interface AsyncAdmin {
    * @param on
    * @return Previous balancer value wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<Boolean> setBalancerOn(boolean on);
+  CompletableFuture<Boolean> balancerSwitch(boolean on);
 
   /**
    * Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the
@@ -993,21 +1017,21 @@ public interface AsyncAdmin {
    * @return true if the balance switch is on, false otherwise. The return value will be wrapped by a
    *         {@link CompletableFuture}.
    */
-  CompletableFuture<Boolean> isBalancerOn();
+  CompletableFuture<Boolean> isBalancerEnabled();
 
   /**
    * Set region normalizer on/off.
    * @param on whether normalizer should be on or off
    * @return Previous normalizer value wrapped by a {@link CompletableFuture}
    */
-  CompletableFuture<Boolean> setNormalizerOn(boolean on);
+  CompletableFuture<Boolean> normalizerSwitch(boolean on);
 
   /**
    * Query the current state of the region normalizer
    * @return true if region normalizer is on, false otherwise. The return value will be wrapped by a
    *         {@link CompletableFuture}
    */
-  CompletableFuture<Boolean> isNormalizerOn();
+  CompletableFuture<Boolean> isNormalizerEnabled();
 
   /**
    * Invoke region normalizer. Can NOT run for various reasons. Check logs.
@@ -1021,14 +1045,14 @@ public interface AsyncAdmin {
    * @param on
    * @return Previous cleaner state wrapped by a {@link CompletableFuture}
    */
-  CompletableFuture<Boolean> setCleanerChoreOn(boolean on);
+  CompletableFuture<Boolean> cleanerChoreSwitch(boolean on);
 
   /**
    * Query the current state of the cleaner chore.
    * @return true if cleaner chore is on, false otherwise. The return value will be wrapped by
    *         a {@link CompletableFuture}
    */
-  CompletableFuture<Boolean> isCleanerChoreOn();
+  CompletableFuture<Boolean> isCleanerChoreEnabled();
 
   /**
    * Ask for cleaner chore to run.
@@ -1042,14 +1066,14 @@ public interface AsyncAdmin {
    * @param on
    * @return the previous state wrapped by a {@link CompletableFuture}
    */
-  CompletableFuture<Boolean> setCatalogJanitorOn(boolean on);
+  CompletableFuture<Boolean> catalogJanitorSwitch(boolean on);
 
   /**
    * Query on the catalog janitor state.
    * @return true if the catalog janitor is on, false otherwise. The return value will be
    *         wrapped by a {@link CompletableFuture}
    */
-  CompletableFuture<Boolean> isCatalogJanitorOn();
+  CompletableFuture<Boolean> isCatalogJanitorEnabled();
 
   /**
    * Ask for a scan of the catalog table.

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index ab529a8..3fe7951 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -84,14 +84,19 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<List<TableDescriptor>> listTables(boolean includeSysTables) {
-    return wrap(rawAdmin.listTables(includeSysTables));
+  public CompletableFuture<List<TableDescriptor>> listTableDescriptors(boolean includeSysTables) {
+    return wrap(rawAdmin.listTableDescriptors(includeSysTables));
   }
 
   @Override
-  public CompletableFuture<List<TableDescriptor>> listTables(Pattern pattern,
+  public CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern,
       boolean includeSysTables) {
-    return wrap(rawAdmin.listTables(pattern, includeSysTables));
+    return wrap(rawAdmin.listTableDescriptors(pattern, includeSysTables));
+  }
+
+  @Override
+  public CompletableFuture<List<TableDescriptor>> listTableDescriptorsByNamespace(String name) {
+    return wrap(rawAdmin.listTableDescriptorsByNamespace(name));
   }
 
   @Override
@@ -106,8 +111,13 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<TableDescriptor> getTableDescriptor(TableName tableName) {
-    return wrap(rawAdmin.getTableDescriptor(tableName));
+  public CompletableFuture<List<TableName>> listTableNamesByNamespace(String name) {
+    return wrap(rawAdmin.listTableNamesByNamespace(name));
+  }
+
+  @Override
+  public CompletableFuture<TableDescriptor> getDescriptor(TableName tableName) {
+    return wrap(rawAdmin.getDescriptor(tableName));
   }
 
   @Override
@@ -214,13 +224,13 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<List<RegionInfo>> getOnlineRegions(ServerName serverName) {
-    return wrap(rawAdmin.getOnlineRegions(serverName));
+  public CompletableFuture<List<RegionInfo>> getRegions(ServerName serverName) {
+    return wrap(rawAdmin.getRegions(serverName));
   }
 
   @Override
-  public CompletableFuture<List<RegionInfo>> getTableRegions(TableName tableName) {
-    return wrap(rawAdmin.getTableRegions(tableName));
+  public CompletableFuture<List<RegionInfo>> getRegions(TableName tableName) {
+    return wrap(rawAdmin.getRegions(tableName));
   }
 
   @Override
@@ -284,23 +294,23 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> setMergeOn(boolean on) {
-    return wrap(rawAdmin.setMergeOn(on));
+  public CompletableFuture<Boolean> mergeSwitch(boolean on) {
+    return wrap(rawAdmin.mergeSwitch(on));
   }
 
   @Override
-  public CompletableFuture<Boolean> isMergeOn() {
-    return wrap(rawAdmin.isMergeOn());
+  public CompletableFuture<Boolean> isMergeEnabled() {
+    return wrap(rawAdmin.isMergeEnabled());
   }
 
   @Override
-  public CompletableFuture<Boolean> setSplitOn(boolean on) {
-    return wrap(rawAdmin.setSplitOn(on));
+  public CompletableFuture<Boolean> splitSwitch(boolean on) {
+    return wrap(rawAdmin.splitSwitch(on));
   }
 
   @Override
-  public CompletableFuture<Boolean> isSplitOn() {
-    return wrap(rawAdmin.isSplitOn());
+  public CompletableFuture<Boolean> isSplitEnabled() {
+    return wrap(rawAdmin.isSplitEnabled());
   }
 
   @Override
@@ -512,9 +522,9 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<byte[]> execProcedureWithRet(String signature, String instance,
+  public CompletableFuture<byte[]> execProcedureWithReturn(String signature, String instance,
       Map<String, String> props) {
-    return wrap(rawAdmin.execProcedureWithRet(signature, instance, props));
+    return wrap(rawAdmin.execProcedureWithReturn(signature, instance, props));
   }
 
   @Override
@@ -643,8 +653,8 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> setBalancerOn(boolean on) {
-    return wrap(rawAdmin.setBalancerOn(on));
+  public CompletableFuture<Boolean> balancerSwitch(boolean on) {
+    return wrap(rawAdmin.balancerSwitch(on));
   }
 
   @Override
@@ -653,18 +663,18 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> isBalancerOn() {
-    return wrap(rawAdmin.isBalancerOn());
+  public CompletableFuture<Boolean> isBalancerEnabled() {
+    return wrap(rawAdmin.isBalancerEnabled());
   }
 
   @Override
-  public CompletableFuture<Boolean> setNormalizerOn(boolean on) {
-    return wrap(rawAdmin.setNormalizerOn(on));
+  public CompletableFuture<Boolean> normalizerSwitch(boolean on) {
+    return wrap(rawAdmin.normalizerSwitch(on));
   }
 
   @Override
-  public CompletableFuture<Boolean> isNormalizerOn() {
-    return wrap(rawAdmin.isNormalizerOn());
+  public CompletableFuture<Boolean> isNormalizerEnabled() {
+    return wrap(rawAdmin.isNormalizerEnabled());
   }
 
   @Override
@@ -673,13 +683,13 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> setCleanerChoreOn(boolean enabled) {
-    return wrap(rawAdmin.setCleanerChoreOn(enabled));
+  public CompletableFuture<Boolean> cleanerChoreSwitch(boolean enabled) {
+    return wrap(rawAdmin.cleanerChoreSwitch(enabled));
   }
 
   @Override
-  public CompletableFuture<Boolean> isCleanerChoreOn() {
-    return wrap(rawAdmin.isCleanerChoreOn());
+  public CompletableFuture<Boolean> isCleanerChoreEnabled() {
+    return wrap(rawAdmin.isCleanerChoreEnabled());
   }
 
   @Override
@@ -688,13 +698,13 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> setCatalogJanitorOn(boolean enabled) {
-    return wrap(rawAdmin.setCatalogJanitorOn(enabled));
+  public CompletableFuture<Boolean> catalogJanitorSwitch(boolean enabled) {
+    return wrap(rawAdmin.catalogJanitorSwitch(enabled));
   }
 
   @Override
-  public CompletableFuture<Boolean> isCatalogJanitorOn() {
-    return wrap(rawAdmin.isCatalogJanitorOn());
+  public CompletableFuture<Boolean> isCatalogJanitorEnabled() {
+    return wrap(rawAdmin.isCatalogJanitorEnabled());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
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 e153381..7669eb2 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
@@ -30,11 +30,11 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
@@ -225,8 +225,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
 public class HBaseAdmin implements Admin {
   private static final Log LOG = LogFactory.getLog(HBaseAdmin.class);
 
-  private static final String ZK_IDENTIFIER_PREFIX =  "hbase-admin-on-";
-
   private ClusterConnection connection;
 
   private volatile Configuration conf;
@@ -1230,14 +1228,17 @@ public class HBaseAdmin implements Admin {
     compactRegion(regionName, columnFamily, false);
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
-  public void compactRegionServer(final ServerName sn, boolean major)
-      throws IOException, InterruptedException {
-    for (RegionInfo region : getRegions(sn)) {
-      compact(this.connection.getAdmin(sn), region, major, null);
+  public void compactRegionServer(final ServerName serverName) throws IOException {
+    for (RegionInfo region : getRegions(serverName)) {
+      compact(this.connection.getAdmin(serverName), region, false, null);
+    }
+  }
+
+  @Override
+  public void majorCompactRegionServer(final ServerName serverName) throws IOException {
+    for (RegionInfo region : getRegions(serverName)) {
+      compact(this.connection.getAdmin(serverName), region, true, null);
     }
   }
 
@@ -2085,21 +2086,11 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public Map<byte[], RegionLoad> getRegionLoad(final ServerName sn) throws IOException {
-    return getRegionLoad(sn, null);
-  }
-
-  @Override
-  public Map<byte[], RegionLoad> getRegionLoad(final ServerName sn, final TableName tableName)
+  public List<RegionLoad> getRegionLoads(ServerName serverName, TableName tableName)
       throws IOException {
-    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
     HBaseRpcController controller = rpcControllerFactory.newController();
-    List<RegionLoad> regionLoads = ProtobufUtil.getRegionLoad(controller, admin, tableName);
-    Map<byte[], RegionLoad> resultMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for (RegionLoad regionLoad : regionLoads) {
-      resultMap.put(regionLoad.getName(), regionLoad);
-    }
-    return resultMap;
+    return ProtobufUtil.getRegionLoad(controller, admin, tableName);
   }
 
   @Override
@@ -3035,6 +3026,18 @@ public class HBaseAdmin implements Admin {
     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)) {
+      Iterator<QuotaSettings> iterator = retriever.iterator();
+      while (iterator.hasNext()) {
+        quotas.add(iterator.next());
+      }
+    }
+    return quotas;
+  }
+
   private <C extends RetryingCallable<V> & Closeable, V> V executeCallable(C callable)
       throws IOException {
     return executeCallable(callable, rpcCallerFactory, operationTimeout, rpcTimeout);
@@ -3793,34 +3796,46 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public boolean[] splitOrMergeEnabledSwitch(final boolean enabled, final boolean synchronous,
-                                          final MasterSwitchType... switchTypes)
-    throws IOException {
-    return executeCallable(new MasterCallable<boolean[]>(getConnection(),
-        getRpcControllerFactory()) {
+  public boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException {
+    return splitOrMergeSwitch(enabled, synchronous, MasterSwitchType.SPLIT);
+  }
+
+  @Override
+  public boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException {
+    return splitOrMergeSwitch(enabled, synchronous, MasterSwitchType.MERGE);
+  }
+
+  private boolean splitOrMergeSwitch(boolean enabled, boolean synchronous,
+      MasterSwitchType switchType) throws IOException {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      protected boolean[] rpcCall() throws Exception {
-        MasterProtos.SetSplitOrMergeEnabledResponse response =
-            master.setSplitOrMergeEnabled(getRpcController(),
-                RequestConverter.buildSetSplitOrMergeEnabledRequest(enabled, synchronous,
-                    switchTypes));
-        boolean[] result = new boolean[switchTypes.length];
-        int i = 0;
-        for (Boolean prevValue : response.getPrevValueList()) {
-          result[i++] = prevValue;
-        }
-        return result;
+      protected Boolean rpcCall() throws Exception {
+        MasterProtos.SetSplitOrMergeEnabledResponse response = master.setSplitOrMergeEnabled(
+          getRpcController(),
+          RequestConverter.buildSetSplitOrMergeEnabledRequest(enabled, synchronous, switchType));
+        return response.getPrevValueList().get(0);
+      }
+    });
+  }
+
+  @Override
+  public boolean isSplitEnabled() throws IOException {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
+      @Override
+      protected Boolean rpcCall() throws Exception {
+        return master.isSplitOrMergeEnabled(getRpcController(),
+          RequestConverter.buildIsSplitOrMergeEnabledRequest(MasterSwitchType.SPLIT)).getEnabled();
       }
     });
   }
 
   @Override
-  public boolean splitOrMergeEnabledSwitch(final MasterSwitchType switchType) throws IOException {
+  public boolean isMergeEnabled() throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
       protected Boolean rpcCall() throws Exception {
         return master.isSplitOrMergeEnabled(getRpcController(),
-          RequestConverter.buildIsSplitOrMergeEnabledRequest(switchType)).getEnabled();
+          RequestConverter.buildIsSplitOrMergeEnabledRequest(MasterSwitchType.MERGE)).getEnabled();
       }
     });
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 6366cf0..fe1d685 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -184,6 +184,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecomm
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
@@ -399,7 +403,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<List<TableDescriptor>> listTables(boolean includeSysTables) {
+  public CompletableFuture<List<TableDescriptor>> listTableDescriptors(boolean includeSysTables) {
     return getTableDescriptors(RequestConverter.buildGetTableDescriptorsRequest(null,
       includeSysTables));
   }
@@ -408,7 +412,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
    * {@link #listTables(boolean)}
    */
   @Override
-  public CompletableFuture<List<TableDescriptor>> listTables(Pattern pattern,
+  public CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern,
       boolean includeSysTables) {
     Preconditions.checkNotNull(pattern,
       "pattern is null. If you don't specify a pattern, use listTables(boolean) instead");
@@ -450,7 +454,31 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<TableDescriptor> getTableDescriptor(TableName tableName) {
+  public CompletableFuture<List<TableDescriptor>> listTableDescriptorsByNamespace(String name) {
+    return this.<List<TableDescriptor>> newMasterCaller().action((controller, stub) -> this
+        .<ListTableDescriptorsByNamespaceRequest, ListTableDescriptorsByNamespaceResponse,
+        List<TableDescriptor>> call(
+          controller, stub,
+          ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name).build(),
+          (s, c, req, done) -> s.listTableDescriptorsByNamespace(c, req, done),
+          (resp) -> ProtobufUtil.toTableDescriptorList(resp)))
+        .call();
+  }
+
+  @Override
+  public CompletableFuture<List<TableName>> listTableNamesByNamespace(String name) {
+    return this.<List<TableName>> newMasterCaller().action((controller, stub) -> this
+        .<ListTableNamesByNamespaceRequest, ListTableNamesByNamespaceResponse,
+        List<TableName>> call(
+          controller, stub,
+          ListTableNamesByNamespaceRequest.newBuilder().setNamespaceName(name).build(),
+          (s, c, req, done) -> s.listTableNamesByNamespace(c, req, done),
+          (resp) -> ProtobufUtil.toTableNameList(resp.getTableNameList())))
+        .call();
+  }
+
+  @Override
+  public CompletableFuture<TableDescriptor> getDescriptor(TableName tableName) {
     CompletableFuture<TableDescriptor> future = new CompletableFuture<>();
     this.<List<TableSchema>> newMasterCaller()
         .action(
@@ -727,7 +755,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<List<RegionInfo>> getOnlineRegions(ServerName serverName) {
+  public CompletableFuture<List<RegionInfo>> getRegions(ServerName serverName) {
     return this.<List<RegionInfo>> newAdminCaller()
         .action((controller, stub) -> this
             .<GetOnlineRegionRequest, GetOnlineRegionResponse, List<RegionInfo>> adminCall(
@@ -738,7 +766,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<List<RegionInfo>> getTableRegions(TableName tableName) {
+  public CompletableFuture<List<RegionInfo>> getRegions(TableName tableName) {
     if (tableName.equals(META_TABLE_NAME)) {
       return connection.getLocator().getRegionLocation(tableName, null, null, operationTimeoutNs)
           .thenApply(loc -> Collections.singletonList(loc.getRegion()));
@@ -873,7 +901,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   private CompletableFuture<Void> compactRegionServer(ServerName sn, boolean major) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    getOnlineRegions(sn).whenComplete((hRegionInfos, err) -> {
+    getRegions(sn).whenComplete((hRegionInfos, err) -> {
       if (err != null) {
         future.completeExceptionally(err);
         return;
@@ -1043,22 +1071,22 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> setMergeOn(boolean on) {
+  public CompletableFuture<Boolean> mergeSwitch(boolean on) {
     return setSplitOrMergeOn(on, MasterSwitchType.MERGE);
   }
 
   @Override
-  public CompletableFuture<Boolean> isMergeOn() {
+  public CompletableFuture<Boolean> isMergeEnabled() {
     return isSplitOrMergeOn(MasterSwitchType.MERGE);
   }
 
   @Override
-  public CompletableFuture<Boolean> setSplitOn(boolean on) {
+  public CompletableFuture<Boolean> splitSwitch(boolean on) {
     return setSplitOrMergeOn(on, MasterSwitchType.SPLIT);
   }
 
   @Override
-  public CompletableFuture<Boolean> isSplitOn() {
+  public CompletableFuture<Boolean> isSplitEnabled() {
     return isSplitOrMergeOn(MasterSwitchType.SPLIT);
   }
 
@@ -1622,7 +1650,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<List<TableCFs>> listReplicatedTableCFs() {
     CompletableFuture<List<TableCFs>> future = new CompletableFuture<List<TableCFs>>();
-    listTables().whenComplete(
+    listTableDescriptors().whenComplete(
       (tables, error) -> {
         if (!completeExceptionally(future, error)) {
           List<TableCFs> replicatedTableCFs = new ArrayList<>();
@@ -2062,7 +2090,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<byte[]> execProcedureWithRet(String signature, String instance,
+  public CompletableFuture<byte[]> execProcedureWithReturn(String signature, String instance,
       Map<String, String> props) {
     ProcedureDescription proDesc =
         ProtobufUtil.buildProcedureDescription(signature, instance, props);
@@ -2861,7 +2889,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> setBalancerOn(final boolean on) {
+  public CompletableFuture<Boolean> balancerSwitch(final boolean on) {
     return this
         .<Boolean> newMasterCaller()
         .action(
@@ -2883,7 +2911,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> isBalancerOn() {
+  public CompletableFuture<Boolean> isBalancerEnabled() {
     return this
         .<Boolean> newMasterCaller()
         .action(
@@ -2894,7 +2922,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> setNormalizerOn(boolean on) {
+  public CompletableFuture<Boolean> normalizerSwitch(boolean on) {
     return this
         .<Boolean> newMasterCaller()
         .action(
@@ -2906,7 +2934,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> isNormalizerOn() {
+  public CompletableFuture<Boolean> isNormalizerEnabled() {
     return this
         .<Boolean> newMasterCaller()
         .action(
@@ -2929,7 +2957,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> setCleanerChoreOn(boolean enabled) {
+  public CompletableFuture<Boolean> cleanerChoreSwitch(boolean enabled) {
     return this
         .<Boolean> newMasterCaller()
         .action(
@@ -2941,7 +2969,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> isCleanerChoreOn() {
+  public CompletableFuture<Boolean> isCleanerChoreEnabled() {
     return this
         .<Boolean> newMasterCaller()
         .action(
@@ -2965,7 +2993,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> setCatalogJanitorOn(boolean enabled) {
+  public CompletableFuture<Boolean> catalogJanitorSwitch(boolean enabled) {
     return this
         .<Boolean> newMasterCaller()
         .action(
@@ -2977,7 +3005,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> isCatalogJanitorOn() {
+  public CompletableFuture<Boolean> isCatalogJanitorEnabled() {
     return this
         .<Boolean> newMasterCaller()
         .action(
@@ -3125,7 +3153,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   private CompletableFuture<byte[][]> getTableSplits(TableName tableName) {
     CompletableFuture<byte[][]> future = new CompletableFuture<>();
-    getTableRegions(tableName).whenComplete((regions, err2) -> {
+    getRegions(tableName).whenComplete((regions, err2) -> {
       if (err2 != null) {
         future.completeExceptionally(err2);
         return;
@@ -3202,7 +3230,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
           future.completeExceptionally(err);
           return;
         }
-        getTableDescriptor(tableName).whenComplete(
+        getDescriptor(tableName).whenComplete(
           (tableDesc, err1) -> {
             if (err1 != null) {
               future.completeExceptionally(err1);
@@ -3249,7 +3277,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   private CompletableFuture<Void> compareTableWithPeerCluster(TableName tableName,
       TableDescriptor tableDesc, ReplicationPeerDescription peer, AsyncAdmin peerAdmin) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    peerAdmin.getTableDescriptor(tableName).whenComplete(
+    peerAdmin.getDescriptor(tableName).whenComplete(
       (peerTableDesc, err) -> {
         if (err != null) {
           future.completeExceptionally(err);
@@ -3280,7 +3308,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
    */
   private CompletableFuture<Void> setTableReplication(TableName tableName, boolean enableRep) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    getTableDescriptor(tableName).whenComplete(
+    getDescriptor(tableName).whenComplete(
       (tableDesc, err) -> {
         if (err != null) {
           future.completeExceptionally(err);

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index bc6e427..3dd1d47 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -163,6 +163,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTabl
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
@@ -437,6 +438,20 @@ public final class ProtobufUtil {
    * @return a list of TableDescriptor
    */
   public static List<TableDescriptor> toTableDescriptorList(GetTableDescriptorsResponse proto) {
+    if (proto == null) {
+      return new ArrayList<>();
+    }
+    return proto.getTableSchemaList().stream().map(ProtobufUtil::toTableDescriptor)
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * Get a list of TableDescriptor from ListTableDescriptorsByNamespaceResponse protobuf
+   * @param proto the ListTableDescriptorsByNamespaceResponse
+   * @return a list of TableDescriptor
+   */
+  public static List<TableDescriptor>
+      toTableDescriptorList(ListTableDescriptorsByNamespaceResponse proto) {
     if (proto == null) return new ArrayList<>();
     return proto.getTableSchemaList().stream().map(ProtobufUtil::toTableDescriptor)
         .collect(Collectors.toList());

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
new file mode 100644
index 0000000..ced1eb8
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
@@ -0,0 +1,91 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.Closeable;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, SmallTests.class })
+public class TestInterfaceAlign {
+
+  private static final Log LOG = LogFactory.getLog(TestInterfaceAlign.class);
+
+  /**
+   * Test methods name match up
+   */
+  @Test
+  public void testAdminWithAsyncAdmin() {
+    List<String> adminMethodNames = getMethodNames(Admin.class);
+    List<String> asyncAdminMethodNames = getMethodNames(AsyncAdmin.class);
+
+    // Remove some special methods
+    adminMethodNames.remove("getOperationTimeout");
+    adminMethodNames.remove("getConnection");
+    adminMethodNames.remove("getConfiguration");
+    adminMethodNames.removeAll(getMethodNames(Abortable.class));
+    adminMethodNames.removeAll(getMethodNames(Closeable.class));
+    // TODO: Remove this after HBASE-19139
+    adminMethodNames.remove("clearBlockCache");
+
+    adminMethodNames.forEach(method -> {
+      boolean contains = asyncAdminMethodNames.contains(method);
+      if (method.endsWith("Async")) {
+        contains = asyncAdminMethodNames.contains(method.replace("Async", ""));
+      }
+      assertTrue("Admin method " + method + " should in AsyncAdmin too", contains);
+    });
+    asyncAdminMethodNames.forEach(method -> {
+      boolean contains = adminMethodNames.contains(method);
+      if (!contains) {
+        contains = adminMethodNames.contains(method + "Async");
+      }
+      assertTrue("AsyncAdmin method " + method + " should in Admin too", contains);
+    });
+  }
+
+  private <T> List<String> getMethodNames(Class<T> c) {
+    return Arrays.asList(c.getDeclaredMethods()).stream().filter(m -> !isDeprecated(m))
+        .map(Method::getName).distinct().collect(Collectors.toList());
+  }
+
+  private boolean isDeprecated(Method method) {
+    Annotation[] annotations = method.getDeclaredAnnotations();
+    for (Annotation annotation : annotations) {
+      if (annotation instanceof Deprecated) {
+        return true;
+      }
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
index 47268c0..da77f29 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
@@ -233,7 +233,7 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
   @Test
   public void testGetRegionLoads() throws Exception {
     // Turn off the balancer
-    admin.setBalancerOn(false).join();
+    admin.balancerSwitch(false).join();
     TableName[] tables =
         new TableName[] { TableName.valueOf(tableName.getNameAsString() + "1"),
             TableName.valueOf(tableName.getNameAsString() + "2"),
@@ -244,13 +244,13 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
     // Check if regions match with the regionLoad from the server
     Collection<ServerName> servers = admin.getRegionServers().get();
     for (ServerName serverName : servers) {
-      List<RegionInfo> regions = admin.getOnlineRegions(serverName).get();
+      List<RegionInfo> regions = admin.getRegions(serverName).get();
       checkRegionsAndRegionLoads(regions, admin.getRegionLoads(serverName).get());
     }
 
     // Check if regionLoad matches the table's regions and nothing is missed
     for (TableName table : tables) {
-      List<RegionInfo> tableRegions = admin.getTableRegions(table).get();
+      List<RegionInfo> tableRegions = admin.getRegions(table).get();
       List<RegionLoad> regionLoads = Lists.newArrayList();
       for (ServerName serverName : servers) {
         regionLoads.addAll(admin.getRegionLoads(serverName, table).get());

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java
index 466ff7f..856a31a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java
@@ -56,7 +56,7 @@ public class TestAsyncDecommissionAdminApi extends TestAsyncAdminBase {
     // leaving one online.
     int i;
     for (i = 0; i < clusterRegionServers.size(); i++) {
-      List<RegionInfo> regionsOnServer = admin.getOnlineRegions(clusterRegionServers.get(i)).get();
+      List<RegionInfo> regionsOnServer = admin.getRegions(clusterRegionServers.get(i)).get();
       if (regionsOnServer.size() > 0) {
         serversToDecommssion.put(clusterRegionServers.get(i), regionsOnServer);
         break;

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java
index 03fd54f..7a2c00f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java
@@ -87,7 +87,7 @@ public class TestAsyncProcedureAdminApi extends TestAsyncAdminBase {
 
   @Test
   public void testExecProcedureWithRet() throws Exception {
-    byte[] result = admin.execProcedureWithRet(SimpleMasterProcedureManager.SIMPLE_SIGNATURE,
+    byte[] result = admin.execProcedureWithReturn(SimpleMasterProcedureManager.SIMPLE_SIGNATURE,
       "myTest2", new HashMap<>()).get();
     assertArrayEquals("Incorrect return data from execProcedure",
       SimpleMasterProcedureManager.SIMPLE_DATA.getBytes(), result);

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
index 1e3af40..8a1afab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
@@ -152,7 +152,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
 
   @Test
   public void testMoveRegion() throws Exception {
-    admin.setBalancerOn(false).join();
+    admin.balancerSwitch(false).join();
 
     RegionInfo hri = createTableAndGetOneRegion(tableName);
     RawAsyncHBaseAdmin rawAdmin = (RawAsyncHBaseAdmin) ASYNC_CONN.getAdmin();
@@ -186,7 +186,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
       }
       Thread.sleep(100);
     }
-    admin.setBalancerOn(true).join();
+    admin.balancerSwitch(true).join();
   }
 
   @Test
@@ -202,7 +202,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
           rs -> {
             ServerName serverName = rs.getServerName();
             try {
-              Assert.assertEquals(admin.getOnlineRegions(serverName).get().size(), rs
+              Assert.assertEquals(admin.getRegions(serverName).get().size(), rs
                   .getRegions().size());
             } catch (Exception e) {
               fail("admin.getOnlineRegions() method throws a exception: " + e.getMessage());
@@ -266,18 +266,18 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
     int originalCount = regionLocations.size();
 
     initSplitMergeSwitch();
-    assertTrue(admin.setSplitOn(false).get());
+    assertTrue(admin.splitSwitch(false).get());
     try {
       admin.split(tableName, Bytes.toBytes(rows / 2)).join();
     } catch (Exception e){
       //Expected
     }
-    int count = admin.getTableRegions(tableName).get().size();
+    int count = admin.getRegions(tableName).get().size();
     assertTrue(originalCount == count);
 
-    assertFalse(admin.setSplitOn(true).get());
+    assertFalse(admin.splitSwitch(true).get());
     admin.split(tableName).join();
-    while ((count = admin.getTableRegions(tableName).get().size()) == originalCount) {
+    while ((count = admin.getRegions(tableName).get().size()) == originalCount) {
       Threads.sleep(100);
     }
     assertTrue(originalCount < count);
@@ -299,36 +299,36 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
     initSplitMergeSwitch();
     admin.split(tableName).join();
     int postSplitCount = originalCount;
-    while ((postSplitCount = admin.getTableRegions(tableName).get().size()) == originalCount) {
+    while ((postSplitCount = admin.getRegions(tableName).get().size()) == originalCount) {
       Threads.sleep(100);
     }
     assertTrue("originalCount=" + originalCount + ", postSplitCount=" + postSplitCount,
       originalCount != postSplitCount);
 
     // Merge switch is off so merge should NOT succeed.
-    assertTrue(admin.setMergeOn(false).get());
-    List<RegionInfo> regions = admin.getTableRegions(tableName).get();
+    assertTrue(admin.mergeSwitch(false).get());
+    List<RegionInfo> regions = admin.getRegions(tableName).get();
     assertTrue(regions.size() > 1);
     admin.mergeRegions(regions.get(0).getRegionName(), regions.get(1).getRegionName(), true).join();
-    int count = admin.getTableRegions(tableName).get().size();
+    int count = admin.getRegions(tableName).get().size();
     assertTrue("postSplitCount=" + postSplitCount + ", count=" + count, postSplitCount == count);
 
     // Merge switch is on so merge should succeed.
-    assertFalse(admin.setMergeOn(true).get());
+    assertFalse(admin.mergeSwitch(true).get());
     admin.mergeRegions(regions.get(0).getRegionName(), regions.get(1).getRegionName(), true).join();
-    count = admin.getTableRegions(tableName).get().size();
+    count = admin.getRegions(tableName).get().size();
     assertTrue((postSplitCount / 2) == count);
   }
 
   private void initSplitMergeSwitch() throws Exception {
-    if (!admin.isSplitOn().get()) {
-      admin.setSplitOn(true).get();
+    if (!admin.isSplitEnabled().get()) {
+      admin.splitSwitch(true).get();
     }
-    if (!admin.isMergeOn().get()) {
-      admin.setMergeOn(true).get();
+    if (!admin.isMergeEnabled().get()) {
+      admin.mergeSwitch(true).get();
     }
-    assertTrue(admin.isSplitOn().get());
-    assertTrue(admin.isMergeOn().get());
+    assertTrue(admin.isSplitEnabled().get());
+    assertTrue(admin.isMergeEnabled().get());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java
index bf60053..4b88bf7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java
@@ -114,13 +114,13 @@ public class TestAsyncReplicationAdminApiWithClusters extends TestAsyncAdminBase
     // default replication scope is local
     createTableWithDefaultConf(tableName);
     admin.enableTableReplication(tableName).join();
-    TableDescriptor tableDesc = admin.getTableDescriptor(tableName).get();
+    TableDescriptor tableDesc = admin.getDescriptor(tableName).get();
     for (ColumnFamilyDescriptor fam : tableDesc.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_GLOBAL, fam.getScope());
     }
 
     admin.disableTableReplication(tableName).join();
-    tableDesc = admin.getTableDescriptor(tableName).get();
+    tableDesc = admin.getDescriptor(tableName).get();
     for (ColumnFamilyDescriptor fam : tableDesc.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_LOCAL, fam.getScope());
     }
@@ -140,7 +140,7 @@ public class TestAsyncReplicationAdminApiWithClusters extends TestAsyncAdminBase
     createTableWithDefaultConf(admin, tableName);
     createTableWithDefaultConf(admin2, tableName);
     TableDescriptorBuilder builder =
-        TableDescriptorBuilder.newBuilder(admin.getTableDescriptor(tableName).get());
+        TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName).get());
     builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("newFamily"))
         .build());
     admin2.disableTable(tableName).join();
@@ -158,7 +158,7 @@ public class TestAsyncReplicationAdminApiWithClusters extends TestAsyncAdminBase
     admin.modifyTable(builder.build()).join();
     admin.enableTable(tableName).join();
     admin.enableTableReplication(tableName).join();
-    TableDescriptor tableDesc = admin.getTableDescriptor(tableName).get();
+    TableDescriptor tableDesc = admin.getDescriptor(tableName).get();
     for (ColumnFamilyDescriptor fam : tableDesc.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_GLOBAL, fam.getScope());
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
index 2ba126f..529346f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
@@ -76,7 +76,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
 
   @Test
   public void testListTables() throws Exception {
-    int numTables = admin.listTables().get().size();
+    int numTables = admin.listTableDescriptors().get().size();
     final TableName tableName1 = TableName.valueOf(tableName.getNameAsString() + "1");
     final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "2");
     final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "3");
@@ -85,7 +85,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
       createTableWithDefaultConf(tables[i]);
     }
 
-    List<TableDescriptor> tableDescs = admin.listTables().get();
+    List<TableDescriptor> tableDescs = admin.listTableDescriptors().get();
     int size = tableDescs.size();
     assertTrue(size >= tables.length);
     for (int i = 0; i < tables.length && i < size; i++) {
@@ -118,7 +118,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
       admin.deleteTable(tables[i]).join();
     }
 
-    tableDescs = admin.listTables(true).get();
+    tableDescs = admin.listTableDescriptors(true).get();
     assertTrue("Not found system tables", tableDescs.size() > 0);
     tableNames = admin.listTableNames(true).get();
     assertTrue("Not found system tables", tableNames.size() > 0);
@@ -134,16 +134,16 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
     TableDescriptor desc = builder.build();
     admin.createTable(desc).join();
     ModifyableTableDescriptor modifyableDesc = ((ModifyableTableDescriptor) desc);
-    TableDescriptor confirmedHtd = admin.getTableDescriptor(tableName).get();
+    TableDescriptor confirmedHtd = admin.getDescriptor(tableName).get();
     assertEquals(modifyableDesc.compareTo((ModifyableTableDescriptor) confirmedHtd), 0);
   }
 
   @Test
   public void testCreateTable() throws Exception {
-    List<TableDescriptor> tables = admin.listTables().get();
+    List<TableDescriptor> tables = admin.listTableDescriptors().get();
     int numTables = tables.size();
     createTableWithDefaultConf(tableName);
-    tables = admin.listTables().get();
+    tables = admin.listTableDescriptors().get();
     assertEquals(numTables + 1, tables.size());
     assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster().getMaster()
         .getTableStateManager().isTableState(tableName, TableState.State.ENABLED));
@@ -650,7 +650,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
     // Modify colymn family
     admin.modifyColumnFamily(tableName, cfd).join();
 
-    TableDescriptor htd = admin.getTableDescriptor(tableName).get();
+    TableDescriptor htd = admin.getDescriptor(tableName).get();
     ColumnFamilyDescriptor hcfd = htd.getColumnFamily(FAMILY_0);
     assertTrue(hcfd.getBlocksize() == newBlockSize);
   }
@@ -720,7 +720,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
   private void verifyTableDescriptor(final TableName tableName, final byte[]... families)
       throws Exception {
     // Verify descriptor from master
-    TableDescriptor htd = admin.getTableDescriptor(tableName).get();
+    TableDescriptor htd = admin.getDescriptor(tableName).get();
     verifyTableDescriptor(htd, tableName, families);
 
     // Verify descriptor from HDFS

http://git-wip-us.apache.org/repos/asf/hbase/blob/52273aa8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncToolAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncToolAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncToolAdminApi.java
index a773188..18d951f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncToolAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncToolAdminApi.java
@@ -35,93 +35,93 @@ public class TestAsyncToolAdminApi extends TestAsyncAdminBase {
 
   @Test
   public void testBalancer() throws Exception {
-    boolean initialState = admin.isBalancerOn().get();
+    boolean initialState = admin.isBalancerEnabled().get();
 
     // Start the balancer, wait for it.
-    boolean prevState = admin.setBalancerOn(!initialState).get();
+    boolean prevState = admin.balancerSwitch(!initialState).get();
 
     // The previous state should be the original state we observed
     assertEquals(initialState, prevState);
 
     // Current state should be opposite of the original
-    assertEquals(!initialState, admin.isBalancerOn().get());
+    assertEquals(!initialState, admin.isBalancerEnabled().get());
 
     // Reset it back to what it was
-    prevState = admin.setBalancerOn(initialState).get();
+    prevState = admin.balancerSwitch(initialState).get();
 
     // 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.isBalancerOn().get());
+    assertEquals(initialState, admin.isBalancerEnabled().get());
   }
 
   @Test
   public void testNormalizer() throws Exception {
-    boolean initialState = admin.isNormalizerOn().get();
+    boolean initialState = admin.isNormalizerEnabled().get();
 
     // flip state
-    boolean prevState = admin.setNormalizerOn(!initialState).get();
+    boolean prevState = admin.normalizerSwitch(!initialState).get();
 
     // The previous state should be the original state we observed
     assertEquals(initialState, prevState);
 
     // Current state should be opposite of the original
-    assertEquals(!initialState, admin.isNormalizerOn().get());
+    assertEquals(!initialState, admin.isNormalizerEnabled().get());
 
     // Reset it back to what it was
-    prevState = admin.setNormalizerOn(initialState).get();
+    prevState = admin.normalizerSwitch(initialState).get();
 
     // 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.isNormalizerOn().get());
+    assertEquals(initialState, admin.isNormalizerEnabled().get());
   }
 
   @Test
   public void testCleanerChore() throws Exception {
-    boolean initialState = admin.isCleanerChoreOn().get();
+    boolean initialState = admin.isCleanerChoreEnabled().get();
 
     // flip state
-    boolean prevState = admin.setCleanerChoreOn(!initialState).get();
+    boolean prevState = admin.cleanerChoreSwitch(!initialState).get();
 
     // The previous state should be the original state we observed
     assertEquals(initialState, prevState);
 
     // Current state should be opposite of the original
-    assertEquals(!initialState, admin.isCleanerChoreOn().get());
+    assertEquals(!initialState, admin.isCleanerChoreEnabled().get());
 
     // Reset it back to what it was
-    prevState = admin.setCleanerChoreOn(initialState).get();
+    prevState = admin.cleanerChoreSwitch(initialState).get();
 
     // 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.isCleanerChoreOn().get());
+    assertEquals(initialState, admin.isCleanerChoreEnabled().get());
   }
 
   @Test
   public void testCatalogJanitor() throws Exception {
-    boolean initialState = admin.isCatalogJanitorOn().get();
+    boolean initialState = admin.isCatalogJanitorEnabled().get();
 
     // flip state
-    boolean prevState = admin.setCatalogJanitorOn(!initialState).get();
+    boolean prevState = admin.catalogJanitorSwitch(!initialState).get();
 
     // The previous state should be the original state we observed
     assertEquals(initialState, prevState);
 
     // Current state should be opposite of the original
-    assertEquals(!initialState, admin.isCatalogJanitorOn().get());
+    assertEquals(!initialState, admin.isCatalogJanitorEnabled().get());
 
     // Reset it back to what it was
-    prevState = admin.setCatalogJanitorOn(initialState).get();
+    prevState = admin.catalogJanitorSwitch(initialState).get();
 
     // 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.isCatalogJanitorOn().get());
+    assertEquals(initialState, admin.isCatalogJanitorEnabled().get());
   }
 }