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/08 10:25:29 UTC

[hbase] branch master updated: HBASE-22007 Add restoreSnapshot and cloneSnapshot with acl methods in AsyncAdmin

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 6e3987c  HBASE-22007 Add restoreSnapshot and cloneSnapshot with acl methods in AsyncAdmin
6e3987c is described below

commit 6e3987cf025421420945b8445c16b32371ff51ad
Author: zhangduo <zh...@apache.org>
AuthorDate: Thu Mar 7 20:52:21 2019 +0800

    HBASE-22007 Add restoreSnapshot and cloneSnapshot with acl methods in AsyncAdmin
    
    Signed-off-by: Zheng Hu <op...@gmail.com>
---
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java |  32 ++-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       |  10 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |  86 ++++----
 ...otWithAcl.java => SnapshotWithAclTestBase.java} | 101 ++++------
 .../hadoop/hbase/client/TestSnapshotWithAcl.java   | 221 ++-------------------
 .../client/TestSnapshotWithAclAsyncAdmin.java      |  58 ++++++
 6 files changed, 194 insertions(+), 314 deletions(-)

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 94dc1e0..5952821 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
@@ -767,14 +767,42 @@ public interface AsyncAdmin {
    * @param snapshotName name of the snapshot to restore
    * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
    */
-  CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot);
+  default CompletableFuture<Void> restoreSnapshot(String snapshotName,
+      boolean takeFailSafeSnapshot) {
+    return restoreSnapshot(snapshotName, takeFailSafeSnapshot, false);
+  }
+
+  /**
+   * Restore the specified snapshot on the original table. (The table must be disabled) If
+   * 'takeFailSafeSnapshot' is set to true, 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 true if the failsafe snapshot should be taken
+   * @param restoreAcl <code>true</code> to restore acl of snapshot
+   */
+  CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot,
+      boolean restoreAcl);
+
+  /**
+   * 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
+   */
+  default CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName) {
+    return 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 restore acl of snapshot
    */
-  CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName);
+  CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName,
+      boolean restoreAcl);
 
   /**
    * List completed snapshots.
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 78c530e..53eaec8 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
@@ -467,13 +467,15 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) {
-    return wrap(rawAdmin.restoreSnapshot(snapshotName, takeFailSafeSnapshot));
+  public CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot,
+      boolean restoreAcl) {
+    return wrap(rawAdmin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl));
   }
 
   @Override
-  public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName) {
-    return wrap(rawAdmin.cloneSnapshot(snapshotName, tableName));
+  public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName,
+      boolean restoreAcl) {
+    return wrap(rawAdmin.cloneSnapshot(snapshotName, tableName, restoreAcl));
   }
 
   @Override
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 04ed3c5..1092332 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
@@ -1852,8 +1852,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> restoreSnapshot(String snapshotName,
-      boolean takeFailSafeSnapshot) {
+  public CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot,
+      boolean restoreAcl) {
     CompletableFuture<Void> future = new CompletableFuture<>();
     addListener(listSnapshots(Pattern.compile(snapshotName)), (snapshotDescriptions, err) -> {
       if (err != null) {
@@ -1881,7 +1881,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         } else if (!exists) {
           // if table does not exist, then just clone snapshot into new table.
           completeConditionalOnFuture(future,
-            internalRestoreSnapshot(snapshotName, finalTableName));
+            internalRestoreSnapshot(snapshotName, finalTableName, restoreAcl));
         } else {
           addListener(isTableDisabled(finalTableName), (disabled, err4) -> {
             if (err4 != null) {
@@ -1890,7 +1890,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
               future.completeExceptionally(new TableNotDisabledException(finalTableName));
             } else {
               completeConditionalOnFuture(future,
-                restoreSnapshot(snapshotName, finalTableName, takeFailSafeSnapshot));
+                restoreSnapshot(snapshotName, finalTableName, takeFailSafeSnapshot, restoreAcl));
             }
           });
         }
@@ -1900,7 +1900,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   private CompletableFuture<Void> restoreSnapshot(String snapshotName, TableName tableName,
-      boolean takeFailSafeSnapshot) {
+      boolean takeFailSafeSnapshot, boolean restoreAcl) {
     if (takeFailSafeSnapshot) {
       CompletableFuture<Void> future = new CompletableFuture<>();
       // Step.1 Take a snapshot of the current state
@@ -1917,40 +1917,42 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
           future.completeExceptionally(err);
         } else {
           // Step.2 Restore snapshot
-          addListener(internalRestoreSnapshot(snapshotName, tableName), (void2, err2) -> {
-            if (err2 != null) {
-              // Step.3.a Something went wrong during the restore and try to rollback.
-              addListener(internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName),
-                (void3, err3) -> {
+          addListener(internalRestoreSnapshot(snapshotName, tableName, restoreAcl),
+            (void2, err2) -> {
+              if (err2 != null) {
+                // Step.3.a Something went wrong during the restore and try to rollback.
+                addListener(
+                  internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName, restoreAcl),
+                  (void3, err3) -> {
+                    if (err3 != null) {
+                      future.completeExceptionally(err3);
+                    } else {
+                      String msg =
+                        "Restore snapshot=" + snapshotName + " failed. Rollback to snapshot=" +
+                          failSafeSnapshotSnapshotName + " succeeded.";
+                      future.completeExceptionally(new RestoreSnapshotException(msg));
+                    }
+                  });
+              } else {
+                // Step.3.b If the restore is succeeded, delete the pre-restore snapshot.
+                LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
+                addListener(deleteSnapshot(failSafeSnapshotSnapshotName), (ret3, err3) -> {
                   if (err3 != null) {
+                    LOG.error(
+                      "Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName,
+                      err3);
                     future.completeExceptionally(err3);
                   } else {
-                    String msg =
-                      "Restore snapshot=" + snapshotName + " failed. Rollback to snapshot=" +
-                        failSafeSnapshotSnapshotName + " succeeded.";
-                    future.completeExceptionally(new RestoreSnapshotException(msg));
+                    future.complete(ret3);
                   }
                 });
-            } else {
-              // Step.3.b If the restore is succeeded, delete the pre-restore snapshot.
-              LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
-              addListener(deleteSnapshot(failSafeSnapshotSnapshotName), (ret3, err3) -> {
-                if (err3 != null) {
-                  LOG.error(
-                    "Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName,
-                    err3);
-                  future.completeExceptionally(err3);
-                } else {
-                  future.complete(ret3);
-                }
-              });
-            }
-          });
+              }
+            });
         }
       });
       return future;
     } else {
-      return internalRestoreSnapshot(snapshotName, tableName);
+      return internalRestoreSnapshot(snapshotName, tableName, restoreAcl);
     }
   }
 
@@ -1966,7 +1968,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName) {
+  public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName,
+      boolean restoreAcl) {
     CompletableFuture<Void> future = new CompletableFuture<>();
     addListener(tableExists(tableName), (exists, err) -> {
       if (err != null) {
@@ -1974,27 +1977,28 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       } else if (exists) {
         future.completeExceptionally(new TableExistsException(tableName));
       } else {
-        completeConditionalOnFuture(future, internalRestoreSnapshot(snapshotName, tableName));
+        completeConditionalOnFuture(future,
+          internalRestoreSnapshot(snapshotName, tableName, restoreAcl));
       }
     });
     return future;
   }
 
-  private CompletableFuture<Void> internalRestoreSnapshot(String snapshotName, TableName tableName) {
+  private CompletableFuture<Void> internalRestoreSnapshot(String snapshotName, TableName tableName,
+      boolean restoreAcl) {
     SnapshotProtos.SnapshotDescription snapshot = SnapshotProtos.SnapshotDescription.newBuilder()
-        .setName(snapshotName).setTable(tableName.getNameAsString()).build();
+      .setName(snapshotName).setTable(tableName.getNameAsString()).build();
     try {
       ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
     } catch (IllegalArgumentException e) {
       return failedFuture(e);
     }
-    return waitProcedureResult(this
-        .<Long> newMasterCaller()
-        .action(
-          (controller, stub) -> this.<RestoreSnapshotRequest, RestoreSnapshotResponse, Long> call(
-            controller, stub, RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot)
-                .setNonceGroup(ng.getNonceGroup()).setNonce(ng.newNonce()).build(), (s, c, req,
-                done) -> s.restoreSnapshot(c, req, done), (resp) -> resp.getProcId())).call());
+    return waitProcedureResult(this.<Long> newMasterCaller().action((controller, stub) -> this
+      .<RestoreSnapshotRequest, RestoreSnapshotResponse, Long> call(controller, stub,
+        RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot).setNonceGroup(ng.getNonceGroup())
+          .setNonce(ng.newNonce()).setRestoreACL(restoreAcl).build(),
+        (s, c, req, done) -> s.restoreSnapshot(c, req, done), (resp) -> resp.getProcId()))
+      .call());
   }
 
   @Override
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAcl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/SnapshotWithAclTestBase.java
similarity index 74%
copy from hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAcl.java
copy to hbase-server/src/test/java/org/apache/hadoop/hbase/client/SnapshotWithAclTestBase.java
index a9d4230..9359fcc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAcl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/SnapshotWithAclTestBase.java
@@ -20,10 +20,7 @@ package org.apache.hadoop.hbase.client;
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Coprocessor;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@@ -33,34 +30,24 @@ import org.apache.hadoop.hbase.security.access.AccessControlLists;
 import org.apache.hadoop.hbase.security.access.AccessController;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.SecureTestUtil;
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.ClassRule;
 import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
-@Category({ MediumTests.class, ClientTests.class })
-public class TestSnapshotWithAcl extends SecureTestUtil {
+public abstract class SnapshotWithAclTestBase extends SecureTestUtil {
 
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestSnapshotWithAcl.class);
-
-  public TableName TEST_TABLE = TableName.valueOf(TEST_UTIL.getRandomUUID().toString());
+  private TableName TEST_TABLE = TableName.valueOf(TEST_UTIL.getRandomUUID().toString());
 
   private static final int ROW_COUNT = 30000;
 
   private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
   private static byte[] TEST_QUALIFIER = Bytes.toBytes("cq");
   private static byte[] TEST_ROW = Bytes.toBytes(0);
-  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static Configuration conf;
-  private static HBaseAdmin admin = null;
+
+  protected static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   // user is table owner. will have all permissions on table
   private static User USER_OWNER;
@@ -83,10 +70,9 @@ public class TestSnapshotWithAcl extends SecureTestUtil {
     public Object run() throws Exception {
       Get g = new Get(TEST_ROW);
       g.addFamily(TEST_FAMILY);
-      try (Connection conn = ConnectionFactory.createConnection(conf)) {
-        try (Table t = conn.getTable(tableName)) {
-          t.get(g);
-        }
+      try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+        Table t = conn.getTable(tableName)) {
+        t.get(g);
       }
       return null;
     }
@@ -103,19 +89,17 @@ public class TestSnapshotWithAcl extends SecureTestUtil {
     public Object run() throws Exception {
       Put p = new Put(TEST_ROW);
       p.addColumn(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(0));
-      try (Connection conn = ConnectionFactory.createConnection(conf)) {
-        try (Table t = conn.getTable(tableName)) {
-          t.put(p);
-        }
+      try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+        Table t = conn.getTable(tableName)) {
+        t.put(p);
       }
       return null;
     }
   }
 
-
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
-    conf = TEST_UTIL.getConfiguration();
+    Configuration conf = TEST_UTIL.getConfiguration();
     // Enable security
     enableSecurity(conf);
     conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName());
@@ -126,7 +110,7 @@ public class TestSnapshotWithAcl extends SecureTestUtil {
     TEST_UTIL.startMiniCluster();
     TEST_UTIL.waitUntilAllRegionsAssigned(AccessControlLists.ACL_TABLE_NAME);
     MasterCoprocessorHost cpHost =
-        TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterCoprocessorHost();
+      TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterCoprocessorHost();
     cpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
 
     USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
@@ -137,24 +121,21 @@ public class TestSnapshotWithAcl extends SecureTestUtil {
 
   @Before
   public void setUp() throws Exception {
-    admin = TEST_UTIL.getHBaseAdmin();
-    HTableDescriptor htd = new HTableDescriptor(TEST_TABLE);
-    HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
-    hcd.setMaxVersions(100);
-    htd.addFamily(hcd);
-    htd.setOwner(USER_OWNER);
-    admin.createTable(htd, new byte[][] { Bytes.toBytes("s") });
+    TEST_UTIL.createTable(TableDescriptorBuilder.newBuilder(TEST_TABLE)
+      .setColumnFamily(
+        ColumnFamilyDescriptorBuilder.newBuilder(TEST_FAMILY).setMaxVersions(100).build())
+      .setOwner(USER_OWNER).build(), new byte[][] { Bytes.toBytes("s") });
     TEST_UTIL.waitTableEnabled(TEST_TABLE);
 
     grantOnTable(TEST_UTIL, USER_RW.getShortName(), TEST_TABLE, TEST_FAMILY, null,
-            Permission.Action.READ, Permission.Action.WRITE);
+      Permission.Action.READ, Permission.Action.WRITE);
 
     grantOnTable(TEST_UTIL, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
-            Permission.Action.READ);
+      Permission.Action.READ);
   }
 
   private void loadData() throws IOException {
-    try (Connection conn = ConnectionFactory.createConnection(conf)) {
+    try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
       try (Table t = conn.getTable(TEST_TABLE)) {
         for (int i = 0; i < ROW_COUNT; i++) {
           Put put = new Put(Bytes.toBytes(i));
@@ -171,21 +152,25 @@ public class TestSnapshotWithAcl extends SecureTestUtil {
   }
 
   private void verifyRows(TableName tableName) throws IOException {
-    try (Connection conn = ConnectionFactory.createConnection(conf)) {
-      try (Table t = conn.getTable(tableName)) {
-        try (ResultScanner scanner = t.getScanner(new Scan())) {
-          Result result;
-          int rowCount = 0;
-          while ((result = scanner.next()) != null) {
-            byte[] value = result.getValue(TEST_FAMILY, TEST_QUALIFIER);
-            Assert.assertArrayEquals(value, Bytes.toBytes(rowCount++));
-          }
-          Assert.assertEquals(ROW_COUNT, rowCount);
-        }
+    try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+      Table t = conn.getTable(tableName); ResultScanner scanner = t.getScanner(new Scan())) {
+      Result result;
+      int rowCount = 0;
+      while ((result = scanner.next()) != null) {
+        byte[] value = result.getValue(TEST_FAMILY, TEST_QUALIFIER);
+        Assert.assertArrayEquals(value, Bytes.toBytes(rowCount++));
       }
+      Assert.assertEquals(ROW_COUNT, rowCount);
     }
   }
 
+  protected abstract void snapshot(String snapshotName, TableName tableName) throws Exception;
+
+  protected abstract void cloneSnapshot(String snapshotName, TableName tableName,
+      boolean restoreAcl) throws Exception;
+
+  protected abstract void restoreSnapshot(String snapshotName, boolean restoreAcl) throws Exception;
+
   @Test
   public void testRestoreSnapshot() throws Exception {
     verifyAllowed(new AccessReadAction(TEST_TABLE), USER_OWNER, USER_RO, USER_RW);
@@ -197,11 +182,11 @@ public class TestSnapshotWithAcl extends SecureTestUtil {
     verifyRows(TEST_TABLE);
 
     String snapshotName1 = TEST_UTIL.getRandomUUID().toString();
-    admin.snapshot(snapshotName1, TEST_TABLE);
+    snapshot(snapshotName1, TEST_TABLE);
 
     // clone snapshot with restoreAcl true.
     TableName tableName1 = TableName.valueOf(TEST_UTIL.getRandomUUID().toString());
-    admin.cloneSnapshot(snapshotName1, tableName1, true);
+    cloneSnapshot(snapshotName1, tableName1, true);
     verifyRows(tableName1);
     verifyAllowed(new AccessReadAction(tableName1), USER_OWNER, USER_RO, USER_RW);
     verifyDenied(new AccessReadAction(tableName1), USER_NONE);
@@ -210,7 +195,7 @@ public class TestSnapshotWithAcl extends SecureTestUtil {
 
     // clone snapshot with restoreAcl false.
     TableName tableName2 = TableName.valueOf(TEST_UTIL.getRandomUUID().toString());
-    admin.cloneSnapshot(snapshotName1, tableName2, false);
+    cloneSnapshot(snapshotName1, tableName2, false);
     verifyRows(tableName2);
     verifyAllowed(new AccessReadAction(tableName2), USER_OWNER);
     verifyDenied(new AccessReadAction(tableName2), USER_NONE, USER_RO, USER_RW);
@@ -226,18 +211,18 @@ public class TestSnapshotWithAcl extends SecureTestUtil {
     verifyDenied(new AccessWriteAction(TEST_TABLE), USER_RO, USER_NONE);
 
     // restore snapshot with restoreAcl false.
-    admin.disableTable(TEST_TABLE);
-    admin.restoreSnapshot(snapshotName1, false, false);
-    admin.enableTable(TEST_TABLE);
+    TEST_UTIL.getAdmin().disableTable(TEST_TABLE);
+    restoreSnapshot(snapshotName1, false);
+    TEST_UTIL.getAdmin().enableTable(TEST_TABLE);
     verifyAllowed(new AccessReadAction(TEST_TABLE), USER_OWNER, USER_RW);
     verifyDenied(new AccessReadAction(TEST_TABLE), USER_RO, USER_NONE);
     verifyAllowed(new AccessWriteAction(TEST_TABLE), USER_OWNER, USER_RW);
     verifyDenied(new AccessWriteAction(TEST_TABLE), USER_RO, USER_NONE);
 
     // restore snapshot with restoreAcl true.
-    admin.disableTable(TEST_TABLE);
-    admin.restoreSnapshot(snapshotName1, false, true);
-    admin.enableTable(TEST_TABLE);
+    TEST_UTIL.getAdmin().disableTable(TEST_TABLE);
+    restoreSnapshot(snapshotName1, true);
+    TEST_UTIL.getAdmin().enableTable(TEST_TABLE);
     verifyAllowed(new AccessReadAction(TEST_TABLE), USER_OWNER, USER_RO, USER_RW);
     verifyDenied(new AccessReadAction(TEST_TABLE), USER_NONE);
     verifyAllowed(new AccessWriteAction(TEST_TABLE), USER_OWNER, USER_RW);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAcl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAcl.java
index a9d4230..614c1fe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAcl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAcl.java
@@ -17,230 +17,33 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.access.AccessControlConstants;
-import org.apache.hadoop.hbase.security.access.AccessControlLists;
-import org.apache.hadoop.hbase.security.access.AccessController;
-import org.apache.hadoop.hbase.security.access.Permission;
-import org.apache.hadoop.hbase.security.access.SecureTestUtil;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
-import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ MediumTests.class, ClientTests.class })
-public class TestSnapshotWithAcl extends SecureTestUtil {
+public class TestSnapshotWithAcl extends SnapshotWithAclTestBase {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestSnapshotWithAcl.class);
+    HBaseClassTestRule.forClass(TestSnapshotWithAcl.class);
 
-  public TableName TEST_TABLE = TableName.valueOf(TEST_UTIL.getRandomUUID().toString());
-
-  private static final int ROW_COUNT = 30000;
-
-  private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
-  private static byte[] TEST_QUALIFIER = Bytes.toBytes("cq");
-  private static byte[] TEST_ROW = Bytes.toBytes(0);
-  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static Configuration conf;
-  private static HBaseAdmin admin = null;
-
-  // user is table owner. will have all permissions on table
-  private static User USER_OWNER;
-  // user with rw permissions on column family.
-  private static User USER_RW;
-  // user with read-only permissions
-  private static User USER_RO;
-  // user with none permissions
-  private static User USER_NONE;
-
-  static class AccessReadAction implements AccessTestAction {
-
-    private TableName tableName;
-
-    public AccessReadAction(TableName tableName) {
-      this.tableName = tableName;
-    }
-
-    @Override
-    public Object run() throws Exception {
-      Get g = new Get(TEST_ROW);
-      g.addFamily(TEST_FAMILY);
-      try (Connection conn = ConnectionFactory.createConnection(conf)) {
-        try (Table t = conn.getTable(tableName)) {
-          t.get(g);
-        }
-      }
-      return null;
-    }
-  }
-
-  static class AccessWriteAction implements AccessTestAction {
-    private TableName tableName;
-
-    public AccessWriteAction(TableName tableName) {
-      this.tableName = tableName;
-    }
-
-    @Override
-    public Object run() throws Exception {
-      Put p = new Put(TEST_ROW);
-      p.addColumn(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(0));
-      try (Connection conn = ConnectionFactory.createConnection(conf)) {
-        try (Table t = conn.getTable(tableName)) {
-          t.put(p);
-        }
-      }
-      return null;
-    }
-  }
-
-
-  @BeforeClass
-  public static void setupBeforeClass() throws Exception {
-    conf = TEST_UTIL.getConfiguration();
-    // Enable security
-    enableSecurity(conf);
-    conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName());
-    // Verify enableSecurity sets up what we require
-    verifyConfiguration(conf);
-    // Enable EXEC permission checking
-    conf.setBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY, true);
-    TEST_UTIL.startMiniCluster();
-    TEST_UTIL.waitUntilAllRegionsAssigned(AccessControlLists.ACL_TABLE_NAME);
-    MasterCoprocessorHost cpHost =
-        TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterCoprocessorHost();
-    cpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
-
-    USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
-    USER_RW = User.createUserForTesting(conf, "rwuser", new String[0]);
-    USER_RO = User.createUserForTesting(conf, "rouser", new String[0]);
-    USER_NONE = User.createUserForTesting(conf, "usernone", new String[0]);
+  @Override
+  protected void snapshot(String snapshotName, TableName tableName) throws Exception {
+    TEST_UTIL.getAdmin().snapshot(snapshotName, tableName);
   }
 
-  @Before
-  public void setUp() throws Exception {
-    admin = TEST_UTIL.getHBaseAdmin();
-    HTableDescriptor htd = new HTableDescriptor(TEST_TABLE);
-    HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
-    hcd.setMaxVersions(100);
-    htd.addFamily(hcd);
-    htd.setOwner(USER_OWNER);
-    admin.createTable(htd, new byte[][] { Bytes.toBytes("s") });
-    TEST_UTIL.waitTableEnabled(TEST_TABLE);
-
-    grantOnTable(TEST_UTIL, USER_RW.getShortName(), TEST_TABLE, TEST_FAMILY, null,
-            Permission.Action.READ, Permission.Action.WRITE);
-
-    grantOnTable(TEST_UTIL, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
-            Permission.Action.READ);
-  }
-
-  private void loadData() throws IOException {
-    try (Connection conn = ConnectionFactory.createConnection(conf)) {
-      try (Table t = conn.getTable(TEST_TABLE)) {
-        for (int i = 0; i < ROW_COUNT; i++) {
-          Put put = new Put(Bytes.toBytes(i));
-          put.addColumn(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i));
-          t.put(put);
-        }
-      }
-    }
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  private void verifyRows(TableName tableName) throws IOException {
-    try (Connection conn = ConnectionFactory.createConnection(conf)) {
-      try (Table t = conn.getTable(tableName)) {
-        try (ResultScanner scanner = t.getScanner(new Scan())) {
-          Result result;
-          int rowCount = 0;
-          while ((result = scanner.next()) != null) {
-            byte[] value = result.getValue(TEST_FAMILY, TEST_QUALIFIER);
-            Assert.assertArrayEquals(value, Bytes.toBytes(rowCount++));
-          }
-          Assert.assertEquals(ROW_COUNT, rowCount);
-        }
-      }
-    }
+  @Override
+  protected void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl)
+      throws Exception {
+    TEST_UTIL.getAdmin().cloneSnapshot(snapshotName, tableName, restoreAcl);
   }
 
-  @Test
-  public void testRestoreSnapshot() throws Exception {
-    verifyAllowed(new AccessReadAction(TEST_TABLE), USER_OWNER, USER_RO, USER_RW);
-    verifyDenied(new AccessReadAction(TEST_TABLE), USER_NONE);
-    verifyAllowed(new AccessWriteAction(TEST_TABLE), USER_OWNER, USER_RW);
-    verifyDenied(new AccessWriteAction(TEST_TABLE), USER_RO, USER_NONE);
-
-    loadData();
-    verifyRows(TEST_TABLE);
-
-    String snapshotName1 = TEST_UTIL.getRandomUUID().toString();
-    admin.snapshot(snapshotName1, TEST_TABLE);
-
-    // clone snapshot with restoreAcl true.
-    TableName tableName1 = TableName.valueOf(TEST_UTIL.getRandomUUID().toString());
-    admin.cloneSnapshot(snapshotName1, tableName1, true);
-    verifyRows(tableName1);
-    verifyAllowed(new AccessReadAction(tableName1), USER_OWNER, USER_RO, USER_RW);
-    verifyDenied(new AccessReadAction(tableName1), USER_NONE);
-    verifyAllowed(new AccessWriteAction(tableName1), USER_OWNER, USER_RW);
-    verifyDenied(new AccessWriteAction(tableName1), USER_RO, USER_NONE);
-
-    // clone snapshot with restoreAcl false.
-    TableName tableName2 = TableName.valueOf(TEST_UTIL.getRandomUUID().toString());
-    admin.cloneSnapshot(snapshotName1, tableName2, false);
-    verifyRows(tableName2);
-    verifyAllowed(new AccessReadAction(tableName2), USER_OWNER);
-    verifyDenied(new AccessReadAction(tableName2), USER_NONE, USER_RO, USER_RW);
-    verifyAllowed(new AccessWriteAction(tableName2), USER_OWNER);
-    verifyDenied(new AccessWriteAction(tableName2), USER_RO, USER_RW, USER_NONE);
-
-    // remove read permission for USER_RO.
-    revokeFromTable(TEST_UTIL, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
-      Permission.Action.READ);
-    verifyAllowed(new AccessReadAction(TEST_TABLE), USER_OWNER, USER_RW);
-    verifyDenied(new AccessReadAction(TEST_TABLE), USER_RO, USER_NONE);
-    verifyAllowed(new AccessWriteAction(TEST_TABLE), USER_OWNER, USER_RW);
-    verifyDenied(new AccessWriteAction(TEST_TABLE), USER_RO, USER_NONE);
-
-    // restore snapshot with restoreAcl false.
-    admin.disableTable(TEST_TABLE);
-    admin.restoreSnapshot(snapshotName1, false, false);
-    admin.enableTable(TEST_TABLE);
-    verifyAllowed(new AccessReadAction(TEST_TABLE), USER_OWNER, USER_RW);
-    verifyDenied(new AccessReadAction(TEST_TABLE), USER_RO, USER_NONE);
-    verifyAllowed(new AccessWriteAction(TEST_TABLE), USER_OWNER, USER_RW);
-    verifyDenied(new AccessWriteAction(TEST_TABLE), USER_RO, USER_NONE);
-
-    // restore snapshot with restoreAcl true.
-    admin.disableTable(TEST_TABLE);
-    admin.restoreSnapshot(snapshotName1, false, true);
-    admin.enableTable(TEST_TABLE);
-    verifyAllowed(new AccessReadAction(TEST_TABLE), USER_OWNER, USER_RO, USER_RW);
-    verifyDenied(new AccessReadAction(TEST_TABLE), USER_NONE);
-    verifyAllowed(new AccessWriteAction(TEST_TABLE), USER_OWNER, USER_RW);
-    verifyDenied(new AccessWriteAction(TEST_TABLE), USER_RO, USER_NONE);
+  @Override
+  protected void restoreSnapshot(String snapshotName, boolean restoreAcl) throws Exception {
+    TEST_UTIL.getAdmin().restoreSnapshot(snapshotName, false, restoreAcl);
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAclAsyncAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAclAsyncAdmin.java
new file mode 100644
index 0000000..792f0e3
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAclAsyncAdmin.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestSnapshotWithAclAsyncAdmin extends SnapshotWithAclTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSnapshotWithAclAsyncAdmin.class);
+
+  @Override
+  protected void snapshot(String snapshotName, TableName tableName) throws Exception {
+    try (AsyncConnection conn =
+      ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get()) {
+      conn.getAdmin().snapshot(snapshotName, tableName).get();
+    }
+  }
+
+  @Override
+  protected void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl)
+      throws Exception {
+    try (AsyncConnection conn =
+      ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get()) {
+      conn.getAdmin().cloneSnapshot(snapshotName, tableName, restoreAcl).get();
+    }
+  }
+
+  @Override
+  protected void restoreSnapshot(String snapshotName, boolean restoreAcl) throws Exception {
+    try (AsyncConnection conn =
+      ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get()) {
+      conn.getAdmin().restoreSnapshot(snapshotName, false, restoreAcl).get();
+    }
+  }
+}