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/11 07:10:09 UTC

[hbase] branch master updated: HBASE-22001 Polish the 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 10ca598  HBASE-22001 Polish the Admin interface
10ca598 is described below

commit 10ca598004ae7db38945145b232a1c809e4ef6b1
Author: zhangduo <zh...@apache.org>
AuthorDate: Sat Mar 9 07:52:58 2019 +0800

    HBASE-22001 Polish the Admin interface
    
    Signed-off-by: stack <st...@apache.org>
---
 .../java/org/apache/hadoop/hbase/client/Admin.java | 480 +++++++++++++--------
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java | 306 ++-----------
 .../hadoop/hbase/client/TestInterfaceAlign.java    |  11 +-
 .../org/apache/hadoop/hbase/util/FutureUtils.java  |  21 +
 .../master/cleaner/TestSnapshotFromMaster.java     |   8 +-
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   | 112 +----
 6 files changed, 380 insertions(+), 558 deletions(-)

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 99db7d5..a0c5401 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
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.util.FutureUtils.get;
+
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collection;
@@ -25,6 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
@@ -41,6 +44,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@@ -58,6 +62,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;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -74,8 +79,28 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.Public
 public interface Admin extends Abortable, Closeable {
+
+  /**
+   * Return the operation timeout for a rpc call.
+   * @see #getSyncWaitTimeout()
+   */
   int getOperationTimeout();
 
+  /**
+   * Return the blocking wait time for an asynchronous operation. Can be configured by
+   * {@code hbase.client.sync.wait.timeout.msec}.
+   * <p/>
+   * For several operations, such as createTable, deleteTable, etc, the rpc call will finish right
+   * after we schedule a procedure at master side, so the timeout will not be controlled by the
+   * above {@link #getOperationTimeout()}. And timeout value here tells you how much time we will
+   * wait until the procedure at master side is finished.
+   * <p/>
+   * In general, you can consider that the implementation for XXXX method is just a
+   * XXXXAsync().get(getSyncWaitTimeout(), TimeUnit.MILLISECONDS).
+   * @see #getOperationTimeout()
+   */
+  int getSyncWaitTimeout();
+
   @Override
   void abort(String why, Throwable e);
 
@@ -136,7 +161,9 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if a remote or network exception occurs
    * @see #listTables()
    */
-  List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException;
+  default List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException {
+    return listTableDescriptors(pattern, false);
+  }
 
   /**
    * List all the userspace tables matching the given regular expression.
@@ -208,7 +235,9 @@ public interface Admin extends Abortable, Closeable {
    * @return array of table names
    * @throws IOException if a remote or network exception occurs
    */
-  TableName[] listTableNames(Pattern pattern) throws IOException;
+  default TableName[] listTableNames(Pattern pattern) throws IOException {
+    return listTableNames(pattern, false);
+  }
 
   /**
    * List all of the names of userspace tables.
@@ -315,7 +344,9 @@ public interface Admin extends Abortable, Closeable {
    * threads, the table may have been created between test-for-existence and attempt-at-creation).
    * @throws IOException
    */
-  void createTable(TableDescriptor desc, byte[][] splitKeys) throws IOException;
+  default void createTable(TableDescriptor desc, byte[][] splitKeys) throws IOException {
+    get(createTableAsync(desc, splitKeys), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Creates a new table but does not block and wait for it to come online.
@@ -337,11 +368,12 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Deletes a table. Synchronous operation.
-   *
    * @param tableName name of table to delete
    * @throws IOException if a remote or network exception occurs
    */
-  void deleteTable(TableName tableName) throws IOException;
+  default void deleteTable(TableName tableName) throws IOException {
+    get(deleteTableAsync(tableName), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Deletes the table but does not block and wait for it to be completely removed.
@@ -403,8 +435,9 @@ public interface Admin extends Abortable, Closeable {
    * @param preserveSplits <code>true</code> if the splits should be preserved
    * @throws IOException if a remote or network exception occurs
    */
-  void truncateTable(TableName tableName, boolean preserveSplits)
-      throws IOException;
+  default void truncateTable(TableName tableName, boolean preserveSplits) throws IOException {
+    get(truncateTableAsync(tableName, preserveSplits), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Truncate the table but does not block and wait for it to be completely enabled. You can use
@@ -421,19 +454,20 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
-   * Enable a table.  May timeout.  Use {@link #enableTableAsync(org.apache.hadoop.hbase.TableName)}
+   * Enable a table. May timeout. Use {@link #enableTableAsync(org.apache.hadoop.hbase.TableName)}
    * and {@link #isTableEnabled(org.apache.hadoop.hbase.TableName)} instead. The table has to be in
    * disabled state for it to be enabled.
-   *
    * @param tableName name of the table
    * @throws IOException if a remote or network exception occurs There could be couple types of
-   * IOException TableNotFoundException means the table doesn't exist. TableNotDisabledException
-   * means the table isn't in disabled state.
+   *           IOException TableNotFoundException means the table doesn't exist.
+   *           TableNotDisabledException means the table isn't in disabled state.
    * @see #isTableEnabled(org.apache.hadoop.hbase.TableName)
    * @see #disableTable(org.apache.hadoop.hbase.TableName)
    * @see #enableTableAsync(org.apache.hadoop.hbase.TableName)
    */
-  void enableTable(TableName tableName) throws IOException;
+  default void enableTable(TableName tableName) throws IOException {
+    get(enableTableAsync(tableName), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Enable the table but does not block and wait for it to be completely enabled.
@@ -501,16 +535,17 @@ public interface Admin extends Abortable, Closeable {
   Future<Void> disableTableAsync(TableName tableName) throws IOException;
 
   /**
-   * Disable table and wait on completion.  May timeout eventually.  Use {@link
-   * #disableTableAsync(org.apache.hadoop.hbase.TableName)} and
+   * Disable table and wait on completion. May timeout eventually. Use
+   * {@link #disableTableAsync(org.apache.hadoop.hbase.TableName)} and
    * {@link #isTableDisabled(org.apache.hadoop.hbase.TableName)} instead. The table has to be in
    * enabled state for it to be disabled.
-   *
    * @param tableName
    * @throws IOException There could be couple types of IOException TableNotFoundException means the
-   * table doesn't exist. TableNotEnabledException means the table isn't in enabled state.
+   *           table doesn't exist. TableNotEnabledException means the table isn't in enabled state.
    */
-  void disableTable(TableName tableName) throws IOException;
+  default void disableTable(TableName tableName) throws IOException {
+    get(disableTableAsync(tableName), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Disable tables matching the passed in pattern and wait on completion. Warning: Use this method
@@ -638,8 +673,10 @@ public interface Admin extends Abortable, Closeable {
    * @param columnFamily column family descriptor of column family to be added
    * @throws IOException if a remote or network exception occurs
    */
-  void addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
-    throws IOException;
+  default void addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
+      throws IOException {
+    get(addColumnFamilyAsync(tableName, columnFamily), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Add a column family to an existing table. Asynchronous operation.
@@ -680,7 +717,10 @@ public interface Admin extends Abortable, Closeable {
    * @param columnFamily name of column family to be deleted
    * @throws IOException if a remote or network exception occurs
    */
-  void deleteColumnFamily(TableName tableName, byte[] columnFamily) throws IOException;
+  default void deleteColumnFamily(TableName tableName, byte[] columnFamily) throws IOException {
+    get(deleteColumnFamilyAsync(tableName, columnFamily), getSyncWaitTimeout(),
+      TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Delete a column family from a table. Asynchronous operation.
@@ -699,9 +739,9 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
-   * 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.
+   * 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
@@ -723,8 +763,11 @@ public interface Admin extends Abortable, Closeable {
    * @param columnFamily new column family descriptor to use
    * @throws IOException if a remote or network exception occurs
    */
-  void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
-      throws IOException;
+  default void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
+      throws IOException {
+    get(modifyColumnFamilyAsync(tableName, columnFamily), getSyncWaitTimeout(),
+      TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Modify an existing column family on a table. Asynchronous operation.
@@ -1348,8 +1391,7 @@ public interface Admin extends Abortable, Closeable {
    * @param splitPoint the explicit position to split on
    * @throws IOException if a remote or network exception occurs
    */
-  void split(TableName tableName, byte[] splitPoint)
-    throws IOException;
+  void split(TableName tableName, byte[] splitPoint) throws IOException;
 
   /**
    * Split an individual region. Asynchronous operation.
@@ -1370,28 +1412,33 @@ public interface Admin extends Abortable, Closeable {
    * @param splitPoint the explicit position to split on
    * @throws IOException if a remote or network exception occurs
    */
-  Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint)
-    throws IOException;
+  Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException;
 
   /**
    * 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 since 2.0 version and will be removed in 3.0 version. use
+   *             {@link #modifyTable(TableDescriptor)}
    */
   @Deprecated
-  void modifyTable(TableName tableName, TableDescriptor td)
-      throws IOException;
+  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
    */
-  void modifyTable(TableDescriptor td) throws IOException;
+  default void modifyTable(TableDescriptor td) throws IOException {
+    get(modifyTableAsync(td), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Modify an existing table, more IRB friendly version. Asynchronous operation.  This means that
@@ -1410,8 +1457,14 @@ public interface Admin extends Abortable, Closeable {
    *             use {@link #modifyTableAsync(TableDescriptor)}
    */
   @Deprecated
-  Future<Void> modifyTableAsync(TableName tableName, TableDescriptor td)
-      throws IOException;
+  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
@@ -1424,31 +1477,24 @@ public interface Admin extends Abortable, Closeable {
    * @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
-   *     operation to complete
+   *         operation to complete
    */
-  Future<Void> modifyTableAsync(TableDescriptor td)
-      throws IOException;
+  Future<Void> modifyTableAsync(TableDescriptor td) throws IOException;
 
   /**
-   * <p>
    * Shuts down the HBase cluster.
-   * </p>
-   * <p>
+   * <p/>
    * Notice that, a success shutdown call may ends with an error since the remote server has already
    * been shutdown.
-   * </p>
    * @throws IOException if a remote or network exception occurs
    */
   void shutdown() throws IOException;
 
   /**
-   * <p>
    * Shuts down the current HBase master only. Does not shutdown the cluster.
-   * </p>
-   * <p>
+   * <p/>
    * Notice that, a success stopMaster call may ends with an error since the remote server has
    * already been shutdown.
-   * </p>
    * @throws IOException if a remote or network exception occurs
    * @see #shutdown()
    */
@@ -1568,71 +1614,65 @@ public interface Admin extends Abortable, Closeable {
   Configuration getConfiguration();
 
   /**
-   * Create a new namespace. Blocks until namespace has been successfully created or an exception
-   * is thrown.
-   *
+   * Create a new namespace. Blocks until namespace has been successfully created or an exception is
+   * thrown.
    * @param descriptor descriptor which describes the new namespace.
    */
-  void createNamespace(NamespaceDescriptor descriptor)
-  throws IOException;
+  default void createNamespace(NamespaceDescriptor descriptor) throws IOException {
+    get(createNamespaceAsync(descriptor), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Create a new namespace.
-   *
    * @param descriptor descriptor which describes the new namespace
    * @return the result of the async create namespace operation. Use Future.get(long, TimeUnit) to
-   *  wait on the operation to complete.
+   *         wait on the operation to complete.
    */
-  Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor)
-  throws IOException;
+  Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor) throws IOException;
 
   /**
-   * Modify an existing namespace.  Blocks until namespace has been successfully modified or an
+   * Modify an existing namespace. Blocks until namespace has been successfully modified or an
    * exception is thrown.
-   *
    * @param descriptor descriptor which describes the new namespace
    */
-  void modifyNamespace(NamespaceDescriptor descriptor)
-  throws IOException;
+  default void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
+    get(modifyNamespaceAsync(descriptor), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Modify an existing namespace.
-   *
    * @param descriptor descriptor which describes the new namespace
    * @return the result of the async modify namespace operation. Use Future.get(long, TimeUnit) to
-   *  wait on the operation to complete.
+   *         wait on the operation to complete.
    */
-  Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor)
-  throws IOException;
+  Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) throws IOException;
 
   /**
-   * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
-   * Blocks until namespace has been successfully deleted or an
-   * exception is thrown.
-   *
+   * Delete an existing namespace. Only empty namespaces (no tables) can be removed. Blocks until
+   * namespace has been successfully deleted or an exception is thrown.
    * @param name namespace name
    */
-  void deleteNamespace(String name) throws IOException;
+  default void deleteNamespace(String name) throws IOException {
+    get(deleteNamespaceAsync(name), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
-   *
    * @param name namespace name
    * @return the result of the async delete namespace operation. Use Future.get(long, TimeUnit) to
-   *  wait on the operation to complete.
+   *         wait on the operation to complete.
    */
   Future<Void> deleteNamespaceAsync(String name) throws IOException;
 
   /**
    * Get a namespace descriptor by name.
-   *
    * @param name name of namespace descriptor
    * @return A descriptor
    * @throws org.apache.hadoop.hbase.NamespaceNotFoundException
    * @throws IOException if a remote or network exception occurs
    */
   NamespaceDescriptor getNamespaceDescriptor(String name)
-  throws NamespaceNotFoundException, IOException;
+      throws NamespaceNotFoundException, IOException;
 
   /**
    * List available namespace descriptors.
@@ -1657,23 +1697,17 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Get list of table descriptors by namespace.
-   *
    * @param name namespace name
    * @return returns a list of TableDescriptors
-   * @throws IOException
    */
-  List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name)
-      throws IOException;
+  List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException;
 
   /**
    * Get list of table names by namespace.
-   *
    * @param name namespace name
    * @return The list of table names in the namespace
-   * @throws IOException
    */
-  TableName[] listTableNamesByNamespace(String name)
-      throws IOException;
+  TableName[] listTableNamesByNamespace(String name) throws IOException;
 
   /**
    * Get the regions of a given table.
@@ -1739,17 +1773,20 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Abort a procedure.
+   * <p/>
    * Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.
    * @param procId ID of the procedure to abort
    * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
-   * @return <code>true</code> if aborted, <code>false</code> if procedure already completed or does not exist
+   * @return <code>true</code> if aborted, <code>false</code> if procedure already completed or does
+   *         not exist
    * @throws IOException
    * @deprecated Since 2.1.1 -- to be removed.
    */
   @Deprecated
-  boolean abortProcedure(
-      long procId,
-      boolean mayInterruptIfRunning) throws IOException;
+  default boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException {
+    return get(abortProcedureAsync(procId, mayInterruptIfRunning), getSyncWaitTimeout(),
+      TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Abort a procedure but does not block and wait for completion.
@@ -1878,19 +1915,20 @@ public interface Admin extends Abortable, Closeable {
    * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
    * taken. If the table is disabled, an offline snapshot is taken. 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
-   * org.apache.hadoop.hbase.snapshot.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[])}.
-   *
+   * a different type or with different parameters) will fail with a
+   * {@link org.apache.hadoop.hbase.snapshot.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[])}.
    * @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 org.apache.hadoop.hbase.snapshot.SnapshotCreationException if snapshot creation failed
    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
    */
-  void snapshot(String snapshotName, TableName tableName)
-      throws IOException, SnapshotCreationException, IllegalArgumentException;
+  default void snapshot(String snapshotName, TableName tableName)
+      throws IOException, SnapshotCreationException, IllegalArgumentException {
+    snapshot(snapshotName, tableName, SnapshotType.FLUSH);
+  }
 
   /**
    * Create a timestamp consistent snapshot for the given table. Snapshots are considered unique
@@ -1898,15 +1936,19 @@ public interface Admin extends Abortable, Closeable {
    * 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.
    */
-  void snapshot(byte[] snapshotName, TableName tableName)
-      throws IOException, SnapshotCreationException, IllegalArgumentException;
+  @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
@@ -1914,19 +1956,18 @@ public interface Admin extends Abortable, Closeable {
    * different 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[])}.
-   *
    * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
-   * snapshots stored on the cluster
+   *          snapshots stored on the cluster
    * @param tableName name of the table to snapshot
    * @param type type of snapshot to take
    * @throws IOException we fail to reach the master
    * @throws SnapshotCreationException if snapshot creation failed
    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
    */
-  void snapshot(String snapshotName,
-      TableName tableName,
-      SnapshotType type) throws IOException, SnapshotCreationException,
-      IllegalArgumentException;
+  default void snapshot(String snapshotName, TableName tableName, SnapshotType type)
+      throws IOException, SnapshotCreationException, IllegalArgumentException {
+    snapshot(new SnapshotDescription(snapshotName, tableName, type));
+  }
 
   /**
    * Take a snapshot and wait for the server to complete that snapshot (blocking). Only a single
@@ -1935,12 +1976,11 @@ public interface Admin extends Abortable, Closeable {
    * single cluster). 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 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
+   * 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.
-   *
    * @param snapshot snapshot to take
    * @throws IOException or we lose contact with the master.
    * @throws SnapshotCreationException if snapshot failed to be taken
@@ -1961,21 +2001,22 @@ public interface Admin extends Abortable, Closeable {
    * {@link #snapshotAsync(SnapshotDescription)} instead.
    */
   @Deprecated
+  @SuppressWarnings("FutureReturnValueIgnored")
   default void takeSnapshotAsync(SnapshotDescription snapshot)
-  throws IOException, SnapshotCreationException {
+      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
    */
-  void snapshotAsync(SnapshotDescription snapshot) throws IOException, SnapshotCreationException;
+  Future<Void> snapshotAsync(SnapshotDescription snapshot)
+      throws IOException, SnapshotCreationException;
 
   /**
    * Check the current state of the passed snapshot. There are three possible states: <ol>
@@ -1998,26 +2039,29 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * 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.
-   *
+   * "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 org.apache.hadoop.hbase.snapshot.RestoreSnapshotException if snapshot failed to be
-   * restored
+   *           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.
    */
-  void restoreSnapshot(byte[] snapshotName) throws IOException, RestoreSnapshotException;
+  @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.
-   *
+   * "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
@@ -2027,59 +2071,66 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * 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.
-   *
+   * "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.
+   * @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
+   * '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.
    */
-  void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot)
-      throws IOException, RestoreSnapshotException;
+  @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 be
-   * restored. If the restore completes without problem the failsafe snapshot is deleted. The
+   * '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
    */
-  void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot)
-      throws IOException, RestoreSnapshotException;
+  default void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot)
+      throws IOException, RestoreSnapshotException {
+    restoreSnapshot(snapshotName, takeFailSafeSnapshot, false);
+  }
 
   /**
    * 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
+   * '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
@@ -2094,60 +2145,81 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * 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.
    */
-  void cloneSnapshot(byte[] snapshotName, TableName tableName)
-      throws IOException, TableExistsException, RestoreSnapshotException;
+  @Deprecated
+  default void cloneSnapshot(byte[] snapshotName, TableName tableName)
+      throws IOException, TableExistsException, RestoreSnapshotException {
+    cloneSnapshot(Bytes.toString(snapshotName), tableName);
+  }
 
   /**
    * 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
-   * @param restoreAcl <code>true</code> to clone acl into newly created table
    * @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
    */
-  void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl)
-      throws IOException, TableExistsException, RestoreSnapshotException;
+  default void cloneSnapshot(String snapshotName, TableName tableName)
+      throws IOException, TableExistsException, RestoreSnapshotException {
+    cloneSnapshot(snapshotName, tableName, false);
+  }
 
   /**
    * 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
+   * @param restoreAcl <code>true</code> to clone acl into newly created table
    * @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
    */
-  void cloneSnapshot(String snapshotName, TableName tableName)
-      throws IOException, TableExistsException, RestoreSnapshotException;
+  default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl)
+      throws IOException, TableExistsException, RestoreSnapshotException {
+    get(cloneSnapshotAsync(snapshotName, tableName, restoreAcl), getSyncWaitTimeout(),
+      TimeUnit.MILLISECONDS);
+  }
 
   /**
-   * Create a new table by cloning the snapshot content, but does not block
-   * and wait for it to be completely cloned.
-   * 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.
-   *
+   * Create a new table by cloning the snapshot content, but does not block and wait for it to be
+   * completely cloned. 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 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 cloned already exists
-   * @return the result of the async clone snapshot. You can use Future.get(long, TimeUnit)
-   *    to wait on the operation to complete.
+   * @return the result of the async clone snapshot. You can use Future.get(long, TimeUnit) to wait
+   *         on the operation to complete.
+   */
+  default Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName)
+      throws IOException, TableExistsException {
+    return cloneSnapshotAsync(snapshotName, tableName, false);
+  }
+
+  /**
+   * 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
+   * @param restoreAcl <code>true</code> to clone acl into newly created table
+   * @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
    */
-  Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName)
-      throws IOException, TableExistsException;
+  Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl)
+      throws IOException, TableExistsException, RestoreSnapshotException;
 
   /**
    * Execute a distributed procedure on a cluster.
@@ -2183,17 +2255,16 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * 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).
+   *          root ZK node name of the procedure).
    * @param instance The instance name of the procedure. For some procedures, this parameter is
-   * optional.
+   *          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;
+      throws IOException;
 
   /**
    * Check the current state of the specified procedure. There are three possible states: <ol>
@@ -2519,12 +2590,15 @@ public interface Admin extends Abortable, Closeable {
    * @param enabled peer state, true if ENABLED and false if DISABLED
    * @throws IOException if a remote or network exception occurs
    */
-  void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
-      throws IOException;
+  default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+      throws IOException {
+    get(addReplicationPeerAsync(peerId, peerConfig, enabled), getSyncWaitTimeout(),
+      TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Add a new replication peer but does not block and wait for it.
-   * <p>
+   * <p/>
    * 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.
@@ -2558,7 +2632,10 @@ public interface Admin extends Abortable, Closeable {
    * @param peerId a short name that identifies the peer
    * @throws IOException if a remote or network exception occurs
    */
-  void removeReplicationPeer(String peerId) throws IOException;
+  default void removeReplicationPeer(String peerId) throws IOException {
+    get(removeReplicationPeerAsync(peerId), getSyncWaitTimeout(),
+      TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Remove a replication peer but does not block and wait for it.
@@ -2577,7 +2654,9 @@ public interface Admin extends Abortable, Closeable {
    * @param peerId a short name that identifies the peer
    * @throws IOException if a remote or network exception occurs
    */
-  void enableReplicationPeer(String peerId) throws IOException;
+  default void enableReplicationPeer(String peerId) throws IOException {
+    get(enableReplicationPeerAsync(peerId), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Enable a replication peer but does not block and wait for it.
@@ -2596,11 +2675,13 @@ public interface Admin extends Abortable, Closeable {
    * @param peerId a short name that identifies the peer
    * @throws IOException if a remote or network exception occurs
    */
-  void disableReplicationPeer(String peerId) throws IOException;
+  default void disableReplicationPeer(String peerId) throws IOException {
+    get(disableReplicationPeerAsync(peerId), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Disable a replication peer but does not block and wait for it.
-   * <p>
+   * <p/>
    * 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.
@@ -2624,12 +2705,15 @@ public interface Admin extends Abortable, Closeable {
    * @param peerConfig new config for the replication peer
    * @throws IOException if a remote or network exception occurs
    */
-  void updateReplicationPeerConfig(String peerId,
-      ReplicationPeerConfig peerConfig) throws IOException;
+  default void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws IOException {
+    get(updateReplicationPeerConfigAsync(peerId, peerConfig), getSyncWaitTimeout(),
+      TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Update the peerConfig for the specified peer but does not block and wait for it.
-   * <p>
+   * <p/>
    * 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.
@@ -2648,9 +2732,16 @@ public interface Admin extends Abortable, Closeable {
    * @throws ReplicationException if tableCfs has conflict with existing config
    * @throws IOException if a remote or network exception occurs
    */
-  void appendReplicationPeerTableCFs(String id,
-      Map<TableName, List<String>> tableCfs)
-      throws ReplicationException, IOException;
+  default void appendReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs)
+      throws ReplicationException, IOException {
+    if (tableCfs == null) {
+      throw new ReplicationException("tableCfs is null");
+    }
+    ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
+    ReplicationPeerConfig newPeerConfig =
+      ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
+    updateReplicationPeerConfig(id, newPeerConfig);
+  }
 
   /**
    * Remove some table-cfs from config of the specified peer.
@@ -2659,9 +2750,16 @@ public interface Admin extends Abortable, Closeable {
    * @throws ReplicationException if tableCfs has conflict with existing config
    * @throws IOException if a remote or network exception occurs
    */
-  void removeReplicationPeerTableCFs(String id,
-      Map<TableName, List<String>> tableCfs)
-      throws ReplicationException, IOException;
+  default void removeReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs)
+      throws ReplicationException, IOException {
+    if (tableCfs == null) {
+      throw new ReplicationException("tableCfs is null");
+    }
+    ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
+    ReplicationPeerConfig newPeerConfig =
+      ReplicationPeerConfigUtil.removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
+    updateReplicationPeerConfig(id, newPeerConfig);
+  }
 
   /**
    * Return a list of replication peers.
@@ -2684,8 +2782,11 @@ public interface Admin extends Abortable, Closeable {
    * @param state a new state of current cluster
    * @throws IOException if a remote or network exception occurs
    */
-  void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
-      throws IOException;
+  default void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws IOException {
+    get(transitReplicationPeerSyncReplicationStateAsync(peerId, state), getSyncWaitTimeout(),
+      TimeUnit.MILLISECONDS);
+  }
 
   /**
    * Transit current cluster to a new state in a synchronous replication peer. But does not block
@@ -2786,25 +2887,24 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if a remote or network exception occurs
    * @return List of servers that are not cleared
    */
-  List<ServerName> clearDeadServers(final List<ServerName> servers) throws IOException;
+  List<ServerName> clearDeadServers(List<ServerName> servers) throws IOException;
 
   /**
    * Create a new table by cloning the existent table schema.
-   *
    * @param tableName name of the table to be cloned
    * @param newTableName name of the new table where the table will be created
    * @param preserveSplits True if the splits should be preserved
    * @throws IOException if a remote or network exception occurs
    */
-  void cloneTableSchema(final TableName tableName, final TableName newTableName,
-      final boolean preserveSplits) throws IOException;
+  void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits)
+      throws IOException;
 
   /**
    * Switch the rpc throttle enable state.
    * @param enable Set to <code>true</code> to enable, <code>false</code> to disable.
    * @return Previous rpc throttle enabled value
    */
-  boolean switchRpcThrottle(final boolean enable) throws IOException;
+  boolean switchRpcThrottle(boolean enable) throws IOException;
 
   /**
    * Get if the rpc throttle is enabled.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 6a38ead..1bfb7b3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcController;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -45,7 +44,6 @@ import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CacheEvictionStats;
@@ -87,7 +85,6 @@ import org.apache.hadoop.hbase.quotas.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
-import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
@@ -270,6 +267,11 @@ public class HBaseAdmin implements Admin {
     return operationTimeout;
   }
 
+  @Override
+  public int getSyncWaitTimeout() {
+    return syncWaitTimeout;
+  }
+
   HBaseAdmin(ClusterConnection connection) throws IOException {
     this.conf = connection.getConfiguration();
     this.connection = connection;
@@ -335,11 +337,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException {
-    return listTableDescriptors(pattern, false);
-  }
-
-  @Override
   public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
       throws IOException {
     return executeCallable(new MasterCallable<List<TableDescriptor>>(getConnection(),
@@ -362,11 +359,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void modifyTable(TableDescriptor td) throws IOException {
-    get(modifyTableAsync(td), syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> modifyTableAsync(TableDescriptor td) throws IOException {
     ModifyTableResponse response = executeCallable(
       new MasterCallable<ModifyTableResponse>(getConnection(), getRpcControllerFactory()) {
@@ -511,11 +503,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public TableName[] listTableNames(Pattern pattern) throws IOException {
-    return listTableNames(pattern, false);
-  }
-
-  @Override
   public TableName[] listTableNames(String regex) throws IOException {
     return listTableNames(Pattern.compile(regex), false);
   }
@@ -611,38 +598,30 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void createTable(TableDescriptor desc)
-  throws IOException {
+  public void createTable(TableDescriptor desc) throws IOException {
     createTable(desc, null);
   }
 
   @Override
-  public void createTable(TableDescriptor desc, byte [] startKey,
-      byte [] endKey, int numRegions)
-  throws IOException {
-    if(numRegions < 3) {
+  public void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
+      throws IOException {
+    if (numRegions < 3) {
       throw new IllegalArgumentException("Must create at least three regions");
-    } else if(Bytes.compareTo(startKey, endKey) >= 0) {
+    } else if (Bytes.compareTo(startKey, endKey) >= 0) {
       throw new IllegalArgumentException("Start key must be smaller than end key");
     }
     if (numRegions == 3) {
-      createTable(desc, new byte[][]{startKey, endKey});
+      createTable(desc, new byte[][] { startKey, endKey });
       return;
     }
-    byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
-    if(splitKeys == null || splitKeys.length != numRegions - 1) {
+    byte[][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
+    if (splitKeys == null || splitKeys.length != numRegions - 1) {
       throw new IllegalArgumentException("Unable to split key range into enough regions");
     }
     createTable(desc, splitKeys);
   }
 
   @Override
-  public void createTable(final TableDescriptor desc, byte [][] splitKeys)
-      throws IOException {
-    get(createTableAsync(desc, splitKeys), syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> createTableAsync(final TableDescriptor desc, final byte[][] splitKeys)
       throws IOException {
     if (desc.getTableName() == null) {
@@ -712,11 +691,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void deleteTable(final TableName tableName) throws IOException {
-    get(deleteTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> deleteTableAsync(final TableName tableName) throws IOException {
     DeleteTableResponse response = executeCallable(
       new MasterCallable<DeleteTableResponse>(getConnection(), getRpcControllerFactory()) {
@@ -792,12 +766,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void truncateTable(final TableName tableName, final boolean preserveSplits)
-      throws IOException {
-    get(truncateTableAsync(tableName, preserveSplits), syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
       throws IOException {
     TruncateTableResponse response =
@@ -858,12 +826,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void enableTable(final TableName tableName)
-  throws IOException {
-    get(enableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> enableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     EnableTableResponse response = executeCallable(
@@ -923,12 +885,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void disableTable(final TableName tableName)
-  throws IOException {
-    get(disableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     DisableTableResponse response = executeCallable(
@@ -1042,12 +998,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void addColumnFamily(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
-      throws IOException {
-    get(addColumnFamilyAsync(tableName, columnFamily), syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> addColumnFamilyAsync(final TableName tableName,
       final ColumnFamilyDescriptor columnFamily) throws IOException {
     AddColumnResponse response =
@@ -1087,14 +1037,8 @@ public class HBaseAdmin implements Admin {
   @Override
   @Deprecated
   public void deleteColumn(final TableName tableName, final byte[] columnFamily)
-  throws IOException {
-    deleteColumnFamily(tableName, columnFamily);
-  }
-
-  @Override
-  public void deleteColumnFamily(final TableName tableName, final byte[] columnFamily)
       throws IOException {
-    get(deleteColumnFamilyAsync(tableName, columnFamily), syncWaitTimeout, TimeUnit.MILLISECONDS);
+    deleteColumnFamily(tableName, columnFamily);
   }
 
   @Override
@@ -1131,12 +1075,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void modifyColumnFamily(final TableName tableName,
-      final ColumnFamilyDescriptor columnFamily) throws IOException {
-    get(modifyColumnFamilyAsync(tableName, columnFamily), syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> modifyColumnFamilyAsync(final TableName tableName,
       final ColumnFamilyDescriptor columnFamily) throws IOException {
     ModifyColumnResponse response =
@@ -1983,22 +1921,6 @@ public class HBaseAdmin implements Admin {
     splitRegionAsync(regionServerPair.getFirst(), splitPoint);
   }
 
-  @Override
-  public void modifyTable(final TableName tableName, final TableDescriptor td)
-      throws IOException {
-    get(modifyTableAsync(tableName, td), syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
-  public Future<Void> modifyTableAsync(final TableName tableName, final 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);
-  }
-
   private static class ModifyTableFuture extends TableFuture<Void> {
     public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName,
         final ModifyTableResponse response) {
@@ -2229,12 +2151,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void createNamespace(final NamespaceDescriptor descriptor)
-  throws IOException {
-    get(createNamespaceAsync(descriptor), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor)
       throws IOException {
     CreateNamespaceResponse response =
@@ -2256,12 +2172,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void modifyNamespace(final NamespaceDescriptor descriptor)
-  throws IOException {
-    get(modifyNamespaceAsync(descriptor), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor)
       throws IOException {
     ModifyNamespaceResponse response =
@@ -2283,12 +2193,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void deleteNamespace(final String name)
-  throws IOException {
-    get(deleteNamespaceAsync(name), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> deleteNamespaceAsync(final String name)
       throws IOException {
     DeleteNamespaceResponse response =
@@ -2489,44 +2393,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Roll the log writer. I.e. when using a file system based write ahead log,
-   * start writing log messages to a new file.
-   *
-   * Note that when talking to a version 1.0+ HBase deployment, the rolling is asynchronous.
-   * This method will return as soon as the roll is requested and the return value will
-   * always be null. Additionally, the named region server may schedule store flushes at the
-   * request of the wal handling the roll request.
-   *
-   * When talking to a 0.98 or older HBase deployment, the rolling is synchronous and the
-   * return value may be either null or a list of encoded region names.
-   *
-   * @param serverName
-   *          The servername of the regionserver. A server name is made of host,
-   *          port and startcode. This is mandatory. Here is an example:
-   *          <code> host187.example.com,60020,1289493121758</code>
-   * @return a set of {@link HRegionInfo#getEncodedName()} that would allow the wal to
-   *         clean up some underlying files. null if there's nothing to flush.
-   * @throws IOException if a remote or network exception occurs
-   * @throws FailedLogCloseException
-   * @deprecated use {@link #rollWALWriter(ServerName)}
-   */
-  @Deprecated
-  public synchronized byte[][] rollHLogWriter(String serverName)
-      throws IOException, FailedLogCloseException {
-    ServerName sn = ServerName.valueOf(serverName);
-    final RollWALWriterResponse response = rollWALWriterImpl(sn);
-    int regionCount = response.getRegionToFlushCount();
-    if (0 == regionCount) {
-      return null;
-    }
-    byte[][] regionsToFlush = new byte[regionCount][];
-    for (int i = 0; i < regionCount; i++) {
-      regionsToFlush[i] = ProtobufUtil.toBytes(response.getRegionToFlush(i));
-    }
-    return regionsToFlush;
-  }
-
   @Override
   public synchronized void rollWALWriter(ServerName serverName)
       throws IOException, FailedLogCloseException {
@@ -2568,26 +2434,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void snapshot(final String snapshotName,
-                       final TableName tableName) throws IOException,
-      SnapshotCreationException, IllegalArgumentException {
-    snapshot(snapshotName, tableName, SnapshotType.FLUSH);
-  }
-
-  @Override
-  public void snapshot(final byte[] snapshotName, final TableName tableName)
-      throws IOException, SnapshotCreationException, IllegalArgumentException {
-    snapshot(Bytes.toString(snapshotName), tableName, SnapshotType.FLUSH);
-  }
-
-  @Override
-  public void snapshot(final String snapshotName, final TableName tableName,
-      SnapshotType type)
-      throws IOException, SnapshotCreationException, IllegalArgumentException {
-    snapshot(new SnapshotDescription(snapshotName, tableName, type));
-  }
-
-  @Override
   public void snapshot(SnapshotDescription snapshotDesc)
       throws IOException, SnapshotCreationException, IllegalArgumentException {
     // actually take the snapshot
@@ -2632,9 +2478,35 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void snapshotAsync(SnapshotDescription snapshotDesc) throws IOException,
-      SnapshotCreationException {
+  public Future<Void> snapshotAsync(SnapshotDescription snapshotDesc)
+      throws IOException, SnapshotCreationException {
     asyncSnapshot(ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc));
+    return new ProcedureFuture<Void>(this, null) {
+
+      @Override
+      protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutException {
+        waitForState(deadlineTs, new WaitForStateCallable() {
+
+          @Override
+          public void throwInterruptedException() throws InterruptedIOException {
+            throw new InterruptedIOException(
+              "Interrupted while waiting for taking snapshot" + snapshotDesc);
+          }
+
+          @Override
+          public void throwTimeoutException(long elapsedTime) throws TimeoutException {
+            throw new TimeoutException("Snapshot '" + snapshotDesc.getName() +
+              "' wasn't completed in expectedTime:" + elapsedTime + " ms");
+          }
+
+          @Override
+          public boolean checkState(int tries) throws IOException {
+            return isSnapshotFinished(snapshotDesc);
+          }
+        });
+        return null;
+      }
+    };
   }
 
   private SnapshotResponse asyncSnapshot(SnapshotProtos.SnapshotDescription snapshot)
@@ -2688,7 +2560,7 @@ public class HBaseAdmin implements Admin {
     restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
   }
 
-  /*
+  /**
    * Check whether the snapshot exists and contains disabled table
    *
    * @param snapshotName name of the snapshot to restore
@@ -2806,36 +2678,12 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void cloneSnapshot(final byte[] snapshotName, final TableName tableName)
-      throws IOException, TableExistsException, RestoreSnapshotException {
-    cloneSnapshot(Bytes.toString(snapshotName), tableName);
-  }
-
-  @Override
-  public void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl)
-      throws IOException, TableExistsException, RestoreSnapshotException {
+  public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
+      boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException {
     if (tableExists(tableName)) {
       throw new TableExistsException(tableName);
     }
-    get(
-      internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl),
-      Integer.MAX_VALUE,
-      TimeUnit.MILLISECONDS);
-  }
-
-  @Override
-  public void cloneSnapshot(final String snapshotName, final TableName tableName)
-      throws IOException, TableExistsException, RestoreSnapshotException {
-    cloneSnapshot(snapshotName, tableName, false);
-  }
-
-  @Override
-  public Future<Void> cloneSnapshotAsync(final String snapshotName, final TableName tableName)
-      throws IOException, TableExistsException {
-    if (tableExists(tableName)) {
-      throw new TableExistsException(tableName);
-    }
-    return internalRestoreSnapshotAsync(snapshotName, tableName, false);
+    return internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl);
   }
 
   @Override
@@ -3977,13 +3825,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
-      throws IOException {
-    get(addReplicationPeerAsync(peerId, peerConfig, enabled), this.syncWaitTimeout,
-      TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
       boolean enabled) throws IOException {
     AddReplicationPeerResponse response = executeCallable(
@@ -3998,11 +3839,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void removeReplicationPeer(String peerId) throws IOException {
-    get(removeReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> removeReplicationPeerAsync(String peerId) throws IOException {
     RemoveReplicationPeerResponse response =
       executeCallable(new MasterCallable<RemoveReplicationPeerResponse>(getConnection(),
@@ -4018,11 +3854,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void enableReplicationPeer(final String peerId) throws IOException {
-    get(enableReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> enableReplicationPeerAsync(final String peerId) throws IOException {
     EnableReplicationPeerResponse response =
       executeCallable(new MasterCallable<EnableReplicationPeerResponse>(getConnection(),
@@ -4038,11 +3869,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void disableReplicationPeer(final String peerId) throws IOException {
-    get(disableReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> disableReplicationPeerAsync(final String peerId) throws IOException {
     DisableReplicationPeerResponse response =
       executeCallable(new MasterCallable<DisableReplicationPeerResponse>(getConnection(),
@@ -4071,13 +3897,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void updateReplicationPeerConfig(final String peerId,
-      final ReplicationPeerConfig peerConfig) throws IOException {
-    get(updateReplicationPeerConfigAsync(peerId, peerConfig), this.syncWaitTimeout,
-      TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> updateReplicationPeerConfigAsync(final String peerId,
       final ReplicationPeerConfig peerConfig) throws IOException {
     UpdateReplicationPeerConfigResponse response =
@@ -4094,13 +3913,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
-      throws IOException {
-    get(transitReplicationPeerSyncReplicationStateAsync(peerId, state), this.syncWaitTimeout,
-      TimeUnit.MILLISECONDS);
-  }
-
-  @Override
   public Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
       SyncReplicationState state) throws IOException {
     TransitReplicationPeerSyncReplicationStateResponse response =
@@ -4118,32 +3930,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void appendReplicationPeerTableCFs(String id,
-      Map<TableName, List<String>> tableCfs)
-      throws ReplicationException, IOException {
-    if (tableCfs == null) {
-      throw new ReplicationException("tableCfs is null");
-    }
-    ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
-    ReplicationPeerConfig newPeerConfig =
-        ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
-    updateReplicationPeerConfig(id, newPeerConfig);
-  }
-
-  @Override
-  public void removeReplicationPeerTableCFs(String id,
-      Map<TableName, List<String>> tableCfs)
-      throws ReplicationException, IOException {
-    if (tableCfs == null) {
-      throw new ReplicationException("tableCfs is null");
-    }
-    ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
-    ReplicationPeerConfig newPeerConfig =
-        ReplicationPeerConfigUtil.removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
-    updateReplicationPeerConfig(id, newPeerConfig);
-  }
-
-  @Override
   public List<ReplicationPeerDescription> listReplicationPeers() throws IOException {
     return listReplicationPeers((Pattern)null);
   }
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
index ed72ac1..953fba7 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
 import java.io.Closeable;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
 import java.util.Arrays;
 import java.util.List;
 import java.util.stream.Collectors;
@@ -32,17 +33,13 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 @Category({ ClientTests.class, SmallTests.class })
 public class TestInterfaceAlign {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestInterfaceAlign.class);
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestInterfaceAlign.class);
+    HBaseClassTestRule.forClass(TestInterfaceAlign.class);
 
   /**
    * Test methods name match up
@@ -54,6 +51,7 @@ public class TestInterfaceAlign {
 
     // Remove some special methods
     adminMethodNames.remove("getOperationTimeout");
+    adminMethodNames.remove("getSyncWaitTimeout");
     adminMethodNames.remove("getConnection");
     adminMethodNames.remove("getConfiguration");
     adminMethodNames.removeAll(getMethodNames(Abortable.class));
@@ -78,7 +76,8 @@ public class TestInterfaceAlign {
   private <T> List<String> getMethodNames(Class<T> c) {
     // DON'T use the getDeclaredMethods as we want to check the Public APIs only.
     return Arrays.asList(c.getMethods()).stream().filter(m -> !isDeprecated(m))
-        .map(Method::getName).distinct().collect(Collectors.toList());
+      .filter(m -> !Modifier.isStatic(m.getModifiers())).map(Method::getName).distinct()
+      .collect(Collectors.toList());
   }
 
   private boolean isDeprecated(Method method) {
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
index 6f0077c..c43bda6 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
@@ -24,7 +24,10 @@ import java.util.concurrent.CompletionException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.function.BiConsumer;
+import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -136,6 +139,24 @@ public final class FutureUtils {
   }
 
   /**
+   * A helper class for getting the result of a Future, and convert the error to an
+   * {@link IOException}.
+   */
+  public static <T> T get(Future<T> future, long timeout, TimeUnit unit) throws IOException {
+    try {
+      return future.get(timeout, unit);
+    } catch (InterruptedException e) {
+      throw (IOException) new InterruptedIOException().initCause(e);
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      Throwables.propagateIfPossible(cause, IOException.class);
+      throw new IOException(cause);
+    } catch (TimeoutException e) {
+      throw new TimeoutIOException(e);
+    }
+  }
+
+  /**
    * Returns a CompletableFuture that is already completed exceptionally with the given exception.
    */
   public static <T> CompletableFuture<T> failedFuture(Throwable e) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
index cc2ee06..fd183fc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
@@ -26,6 +26,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.Future;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
@@ -313,7 +314,7 @@ public class TestSnapshotFromMaster {
     // take a snapshot of the table
     String snapshotName = "snapshot";
     byte[] snapshotNameBytes = Bytes.toBytes(snapshotName);
-    admin.snapshot(snapshotNameBytes, TABLE_NAME);
+    admin.snapshot(snapshotName, TABLE_NAME);
 
     LOG.info("After snapshot File-System state");
     FSUtils.logFileSystemState(fs, rootDir, LOG);
@@ -436,12 +437,13 @@ public class TestSnapshotFromMaster {
       table.put(put);
     }
     String snapshotName = "testAsyncSnapshotWillNotBlockSnapshotHFileCleaner01";
-    UTIL.getAdmin().snapshotAsync(new org.apache.hadoop.hbase.client.SnapshotDescription(
+    Future<Void> future =
+      UTIL.getAdmin().snapshotAsync(new org.apache.hadoop.hbase.client.SnapshotDescription(
         snapshotName, TABLE_NAME, SnapshotType.FLUSH));
     Waiter.waitFor(UTIL.getConfiguration(), 10 * 1000L, 200L,
       () -> UTIL.getAdmin().listSnapshots(Pattern.compile(snapshotName)).size() == 1);
     assertTrue(master.getSnapshotManager().isTakingAnySnapshot());
-    Thread.sleep(11 * 1000L);
+    future.get();
     assertFalse(master.getSnapshotManager().isTakingAnySnapshot());
   }
 }
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 ccc798f..888256e 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
@@ -26,7 +26,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Future;
 import java.util.regex.Pattern;
-
 import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CacheEvictionStats;
@@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.RegionMetrics;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Admin;
@@ -56,11 +56,11 @@ 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.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.thrift2.ThriftUtilities;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
@@ -79,6 +79,7 @@ public class ThriftAdmin implements Admin {
   private THBaseService.Client client;
   private TTransport transport;
   private int operationTimeout;
+  private int syncWaitTimeout;
   private Configuration conf;
 
 
@@ -86,7 +87,8 @@ public class ThriftAdmin implements Admin {
     this.client = client;
     this.transport = tTransport;
     this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
-        HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
+      HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
+    this.syncWaitTimeout = conf.getInt("hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
     this.conf = conf;
   }
 
@@ -96,8 +98,12 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void abort(String why, Throwable e) {
+  public int getSyncWaitTimeout() {
+    return syncWaitTimeout;
+  }
 
+  @Override
+  public void abort(String why, Throwable e) {
   }
 
   @Override
@@ -987,7 +993,7 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void snapshotAsync(SnapshotDescription snapshot) {
+  public Future<Void> snapshotAsync(SnapshotDescription snapshot) {
     throw new NotImplementedException("snapshotAsync not supported in ThriftAdmin");
 
   }
@@ -1015,44 +1021,14 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot) {
-    throw new NotImplementedException("restoreSnapshot not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) {
-    throw new NotImplementedException("restoreSnapshot not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot,
       boolean restoreAcl) {
     throw new NotImplementedException("restoreSnapshot not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public void cloneSnapshot(byte[] snapshotName, TableName tableName) {
-    throw new NotImplementedException("cloneSnapshot not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl) {
-    throw new NotImplementedException("cloneSnapshot not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public void cloneSnapshot(String snapshotName, TableName tableName) {
-    throw new NotImplementedException("cloneSnapshot not supported in ThriftAdmin");
-
   }
 
   @Override
-  public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName) {
+  public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean cloneAcl)
+      throws IOException, TableExistsException, RestoreSnapshotException {
     throw new NotImplementedException("cloneSnapshotAsync not supported in ThriftAdmin");
   }
 
@@ -1103,43 +1079,36 @@ public class ThriftAdmin implements Admin {
   @Override
   public void deleteSnapshot(byte[] snapshotName) {
     throw new NotImplementedException("deleteSnapshot not supported in ThriftAdmin");
-
   }
 
   @Override
   public void deleteSnapshot(String snapshotName) {
     throw new NotImplementedException("deleteSnapshot not supported in ThriftAdmin");
-
   }
 
   @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");
-
   }
 
   @Override
   public void setQuota(QuotaSettings quota) {
     throw new NotImplementedException("setQuota not supported in ThriftAdmin");
-
   }
 
   @Override
@@ -1165,13 +1134,11 @@ public class ThriftAdmin implements Admin {
   @Override
   public void updateConfiguration(ServerName server) {
     throw new NotImplementedException("updateConfiguration not supported in ThriftAdmin");
-
   }
 
   @Override
   public void updateConfiguration() {
     throw new NotImplementedException("updateConfiguration not supported in ThriftAdmin");
-
   }
 
   @Override
@@ -1200,46 +1167,22 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) {
-    throw new NotImplementedException("addReplicationPeer not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
       boolean enabled) {
     throw new NotImplementedException("addReplicationPeerAsync not supported in ThriftAdmin");
   }
 
   @Override
-  public void removeReplicationPeer(String peerId) {
-    throw new NotImplementedException("removeReplicationPeer not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public Future<Void> removeReplicationPeerAsync(String peerId) {
     throw new NotImplementedException("removeReplicationPeerAsync not supported in ThriftAdmin");
   }
 
   @Override
-  public void enableReplicationPeer(String peerId) {
-    throw new NotImplementedException("enableReplicationPeer not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public Future<Void> enableReplicationPeerAsync(String peerId) {
     throw new NotImplementedException("enableReplicationPeerAsync not supported in ThriftAdmin");
   }
 
   @Override
-  public void disableReplicationPeer(String peerId) {
-    throw new NotImplementedException("disableReplicationPeer not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public Future<Void> disableReplicationPeerAsync(String peerId) {
     throw new NotImplementedException("disableReplicationPeerAsync not supported in ThriftAdmin");
   }
@@ -1250,12 +1193,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) {
-    throw new NotImplementedException("updateReplicationPeerConfig not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public Future<Void> updateReplicationPeerConfigAsync(String peerId,
       ReplicationPeerConfig peerConfig) {
     throw new NotImplementedException(
@@ -1263,20 +1200,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void appendReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs)
-      throws ReplicationException, IOException {
-    throw new NotImplementedException("appendReplicationPeerTableCFs not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public void removeReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs)
-      throws ReplicationException, IOException {
-    throw new NotImplementedException("removeReplicationPeerTableCFs not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public List<ReplicationPeerDescription> listReplicationPeers() {
     throw new NotImplementedException("listReplicationPeers not supported in ThriftAdmin");
   }
@@ -1285,15 +1208,6 @@ public class ThriftAdmin implements Admin {
   public List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) {
     throw new NotImplementedException("listReplicationPeers not supported in ThriftAdmin");
   }
-
-  @Override
-  public void transitReplicationPeerSyncReplicationState(String peerId,
-      SyncReplicationState state) {
-    throw new NotImplementedException(
-        "transitReplicationPeerSyncReplicationState not supported in ThriftAdmin");
-
-  }
-
   @Override
   public Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
       SyncReplicationState state) {