You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2020/04/01 15:38:18 UTC

[hbase] branch master updated: HBASE-24097 [Flakey Tests] TestSnapshotScannerHDFSAclController#testRestoreSnapshot (#1405)

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

stack 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 0595200  HBASE-24097 [Flakey Tests] TestSnapshotScannerHDFSAclController#testRestoreSnapshot (#1405)
0595200 is described below

commit 0595200fb5f23e6f43fb41208b77a34dab0a1ba3
Author: Michael Stack <sa...@users.noreply.github.com>
AuthorDate: Wed Apr 1 08:33:44 2020 -0700

    HBASE-24097 [Flakey Tests] TestSnapshotScannerHDFSAclController#testRestoreSnapshot (#1405)
    
    hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
     Add debug for when assert fails (it fails on occasion locally)
    
    hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestHDFSAclHelper.java
     Move this inner class out standalone since it used now by two tests.
    
    hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java
     Moved out testRestoreSnapshot and made methods in here static so could
     be used by a new adacent test. Also made tablenames unique to methods
     thinking that was root of original issue (wasn't but no harm in doing
     this change) Moved out the inner class TestHDFSAclHelper.
    
    hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController2.java
     New class that sets up same context as
     TestSnapshotScannerHDFSAclController but just to run single
     testRestoreSnapshot method.
    
    hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
     Some debug.
    
    Signed-off-by: Yi Mei
---
 .../replication/TestReplicationTrackerZKImpl.java  |  13 +-
 .../hbase/security/access/TestHDFSAclHelper.java   | 173 +++++++++
 .../TestSnapshotScannerHDFSAclController.java      | 401 ++++++---------------
 .../TestSnapshotScannerHDFSAclController2.java     | 185 ++++++++++
 .../hbase/security/token/TestZKSecretWatcher.java  |   5 +-
 5 files changed, 476 insertions(+), 301 deletions(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index 9d3283d..1500a71 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -116,22 +117,26 @@ public class TestReplicationTrackerZKImpl {
     // 1 region server
     ZKUtil.createWithParents(zkw,
       ZNodePaths.joinZNode(zkw.getZNodePaths().rsZNode, "hostname1.example.org:1234"));
-    assertEquals(1, rt.getListOfRegionServers().size());
+    List<String> rss = rt.getListOfRegionServers();
+    assertEquals(rss.toString(), 1, rss.size());
 
     // 2 region servers
     ZKUtil.createWithParents(zkw,
       ZNodePaths.joinZNode(zkw.getZNodePaths().rsZNode, "hostname2.example.org:1234"));
-    assertEquals(2, rt.getListOfRegionServers().size());
+    rss = rt.getListOfRegionServers();
+    assertEquals(rss.toString(), 2, rss.size());
 
     // 1 region server
     ZKUtil.deleteNode(zkw,
       ZNodePaths.joinZNode(zkw.getZNodePaths().rsZNode, "hostname2.example.org:1234"));
-    assertEquals(1, rt.getListOfRegionServers().size());
+    rss = rt.getListOfRegionServers();
+    assertEquals(1, rss.size());
 
     // 0 region server
     ZKUtil.deleteNode(zkw,
       ZNodePaths.joinZNode(zkw.getZNodePaths().rsZNode, "hostname1.example.org:1234"));
-    assertEquals(0, rt.getListOfRegionServers().size());
+    rss = rt.getListOfRegionServers();
+    assertEquals(rss.toString(), 0, rss.size());
   }
 
   @Test
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestHDFSAclHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestHDFSAclHelper.java
new file mode 100644
index 0000000..420fb97
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestHDFSAclHelper.java
@@ -0,0 +1,173 @@
+/*
+ * 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.security.access;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.*;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+final class TestHDFSAclHelper {
+  private static final Logger LOG = LoggerFactory.getLogger(TestHDFSAclHelper.class);
+
+  private TestHDFSAclHelper() {
+  }
+
+  static void grantOnTable(HBaseTestingUtility util, String user, TableName tableName,
+                           Permission.Action... actions) throws Exception {
+    SecureTestUtil.grantOnTable(util, user, tableName, null, null, actions);
+  }
+
+  static void createNamespace(HBaseTestingUtility util, String namespace) throws IOException {
+    if (Arrays.stream(util.getAdmin().listNamespaceDescriptors())
+        .noneMatch(ns -> ns.getName().equals(namespace))) {
+      NamespaceDescriptor namespaceDescriptor = NamespaceDescriptor.create(namespace).build();
+      util.getAdmin().createNamespace(namespaceDescriptor);
+    }
+  }
+
+  static Table createTable(HBaseTestingUtility util, TableName tableName) throws IOException {
+    createNamespace(util, tableName.getNamespaceAsString());
+    TableDescriptor td = getTableDescriptorBuilder(util, tableName)
+        .setValue(SnapshotScannerHDFSAclHelper.ACL_SYNC_TO_HDFS_ENABLE, "true").build();
+    byte[][] splits = new byte[][] { Bytes.toBytes("2"), Bytes.toBytes("4") };
+    return util.createTable(td, splits);
+  }
+
+  static Table createMobTable(HBaseTestingUtility util, TableName tableName) throws IOException {
+    createNamespace(util, tableName.getNamespaceAsString());
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN1).setMobEnabled(true)
+            .setMobThreshold(0).build())
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN2).setMobEnabled(true)
+            .setMobThreshold(0).build())
+        .setOwner(User.createUserForTesting(util.getConfiguration(), "owner", new String[] {}))
+        .setValue(SnapshotScannerHDFSAclHelper.ACL_SYNC_TO_HDFS_ENABLE, "true").build();
+    byte[][] splits = new byte[][] { Bytes.toBytes("2"), Bytes.toBytes("4") };
+    return util.createTable(td, splits);
+  }
+
+  static TableDescriptor createUserScanSnapshotDisabledTable(HBaseTestingUtility util,
+      TableName tableName) throws IOException {
+    createNamespace(util, tableName.getNamespaceAsString());
+    TableDescriptor td = getTableDescriptorBuilder(util, tableName).build();
+    byte[][] splits = new byte[][] { Bytes.toBytes("2"), Bytes.toBytes("4") };
+    try (Table t = util.createTable(td, splits)) {
+      put(t);
+    }
+    return td;
+  }
+
+  static TableDescriptorBuilder getTableDescriptorBuilder(HBaseTestingUtility util,
+      TableName tableName) {
+    return TableDescriptorBuilder.newBuilder(tableName)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN1).build())
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN2).build())
+        .setOwner(User.createUserForTesting(util.getConfiguration(), "owner", new String[] {}));
+  }
+
+  static void createTableAndPut(HBaseTestingUtility util, TableName tableNam) throws IOException {
+    try (Table t = createTable(util, tableNam)) {
+      put(t);
+    }
+  }
+
+  static final byte[] COLUMN1 = Bytes.toBytes("A");
+  static final byte[] COLUMN2 = Bytes.toBytes("B");
+
+  static void put(Table hTable) throws IOException {
+    List<Put> puts = new ArrayList<>();
+    for (int i = 0; i < 6; i++) {
+      Put put = new Put(Bytes.toBytes(i));
+      put.addColumn(COLUMN1, null, Bytes.toBytes(i));
+      put.addColumn(COLUMN2, null, Bytes.toBytes(i + 1));
+      puts.add(put);
+    }
+    hTable.put(puts);
+  }
+
+  static void put2(Table hTable) throws IOException {
+    List<Put> puts = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      if (i == 5) {
+        continue;
+      }
+      Put put = new Put(Bytes.toBytes(i));
+      put.addColumn(COLUMN1, null, Bytes.toBytes(i + 2));
+      put.addColumn(COLUMN2, null, Bytes.toBytes(i + 3));
+      puts.add(put);
+    }
+    hTable.put(puts);
+  }
+
+  /**
+   * Check if user is able to read expected rows from the specific snapshot
+   * @param user the specific user
+   * @param snapshot the snapshot to be scanned
+   * @param expectedRowCount expected row count read from snapshot, -1 if expects
+   *          AccessControlException
+   * @throws IOException user scan snapshot error
+   * @throws InterruptedException user scan snapshot error
+   */
+  static void canUserScanSnapshot(HBaseTestingUtility util, User user, String snapshot,
+      int expectedRowCount) throws IOException, InterruptedException {
+    PrivilegedExceptionAction<Void> action =
+        getScanSnapshotAction(util.getConfiguration(), snapshot, expectedRowCount);
+    user.runAs(action);
+  }
+
+  static PrivilegedExceptionAction<Void> getScanSnapshotAction(Configuration conf,
+                                                               String snapshotName, long expectedRowCount) {
+    return () -> {
+      try {
+        Path restoreDir = new Path(SnapshotScannerHDFSAclHelper.SNAPSHOT_RESTORE_TMP_DIR_DEFAULT);
+        Scan scan = new Scan();
+        TableSnapshotScanner scanner =
+            new TableSnapshotScanner(conf, restoreDir, snapshotName, scan);
+        int rowCount = 0;
+        while (true) {
+          Result result = scanner.next();
+          if (result == null) {
+            break;
+          }
+          rowCount++;
+        }
+        scanner.close();
+        assertEquals(expectedRowCount, rowCount);
+      } catch (Exception e) {
+        LOG.debug("Scan snapshot error, snapshot {}", snapshotName, e);
+        assertEquals(expectedRowCount, -1);
+      }
+      return null;
+    };
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java
index 990e139..a5f615e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java
@@ -26,13 +26,8 @@ import static org.apache.hadoop.hbase.security.access.SnapshotScannerHDFSAclCont
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -44,20 +39,14 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.client.TableSnapshotScanner;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.junit.AfterClass;
@@ -84,7 +73,7 @@ public class TestSnapshotScannerHDFSAclController {
   private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static Configuration conf = TEST_UTIL.getConfiguration();
   private static Admin admin = null;
-  private static FileSystem fs = null;
+  private static FileSystem FS = null;
   private static Path rootDir = null;
   private static User unGrantUser = null;
   private static SnapshotScannerHDFSAclHelper helper;
@@ -108,9 +97,10 @@ public class TestSnapshotScannerHDFSAclController {
           + SnapshotScannerHDFSAclController.class.getName());
 
     TEST_UTIL.startMiniCluster();
+    TEST_UTIL.waitTableAvailable(PermissionStorage.ACL_TABLE_NAME);
     admin = TEST_UTIL.getAdmin();
     rootDir = TEST_UTIL.getDefaultRootDirPath();
-    fs = rootDir.getFileSystem(conf);
+    FS = rootDir.getFileSystem(conf);
     unGrantUser = User.createUserForTesting(conf, UN_GRANT_USER, new String[] {});
     helper = new SnapshotScannerHDFSAclHelper(conf, admin.getConnection());
 
@@ -120,21 +110,21 @@ public class TestSnapshotScannerHDFSAclController {
           SnapshotScannerHDFSAclHelper.COMMON_DIRECTORY_PERMISSION_DEFAULT));
     Path path = rootDir;
     while (path != null) {
-      fs.setPermission(path, commonDirectoryPermission);
+      FS.setPermission(path, commonDirectoryPermission);
       path = path.getParent();
     }
     // set restore directory permission
     Path restoreDir = new Path(SnapshotScannerHDFSAclHelper.SNAPSHOT_RESTORE_TMP_DIR_DEFAULT);
-    if (!fs.exists(restoreDir)) {
-      fs.mkdirs(restoreDir);
-      fs.setPermission(restoreDir,
+    if (!FS.exists(restoreDir)) {
+      FS.mkdirs(restoreDir);
+      FS.setPermission(restoreDir,
         new FsPermission(
             conf.get(SnapshotScannerHDFSAclHelper.SNAPSHOT_RESTORE_DIRECTORY_PERMISSION,
               SnapshotScannerHDFSAclHelper.SNAPSHOT_RESTORE_DIRECTORY_PERMISSION_DEFAULT)));
     }
     path = restoreDir.getParent();
     while (path != null) {
-      fs.setPermission(path, commonDirectoryPermission);
+      FS.setPermission(path, commonDirectoryPermission);
       path = path.getParent();
     }
 
@@ -154,7 +144,7 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
+    TableName table = TableName.valueOf(namespace, name.getMethodName());
     String snapshot1 = namespace + "s1";
     String snapshot2 = namespace + "s2";
 
@@ -187,9 +177,9 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace1 = name.getMethodName();
-    TableName table1 = TableName.valueOf(namespace1, "t1");
+    TableName table1 = TableName.valueOf(namespace1, name.getMethodName() + ".1");
     String namespace2 = namespace1 + "2";
-    TableName table2 = TableName.valueOf(namespace2, "t2");
+    TableName table2 = TableName.valueOf(namespace2, name.getMethodName() + ".2");
     String snapshot1 = namespace1 + "s1";
     String snapshot2 = namespace2 + "s2";
 
@@ -212,9 +202,9 @@ public class TestSnapshotScannerHDFSAclController {
     assertFalse(hasUserGlobalHdfsAcl(aclTable, grantUserName));
     assertTrue(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace1));
     assertFalse(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace2));
-    checkUserAclEntry(helper.getGlobalRootPaths(), grantUserName, false, false);
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace1), grantUserName, true, true);
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace2), grantUserName, false, false);
+    checkUserAclEntry(FS, helper.getGlobalRootPaths(), grantUserName, false, false);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace1), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace2), grantUserName, false, false);
   }
 
   @Test
@@ -222,8 +212,8 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table1 = TableName.valueOf(namespace, "t1");
-    TableName table2 = TableName.valueOf(namespace, "t2");
+    TableName table1 = TableName.valueOf(namespace, name.getMethodName() + ".1");
+    TableName table2 = TableName.valueOf(namespace, name.getMethodName() + ".2");
     String snapshot1 = namespace + "s1";
     String snapshot2 = namespace + "s2";
     // grant G(R)
@@ -243,9 +233,9 @@ public class TestSnapshotScannerHDFSAclController {
     assertFalse(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace));
     assertTrue(hasUserTableHdfsAcl(aclTable, grantUserName, table1));
     assertFalse(hasUserTableHdfsAcl(aclTable, grantUserName, table2));
-    checkUserAclEntry(helper.getGlobalRootPaths(), grantUserName, false, false);
-    checkUserAclEntry(helper.getTableRootPaths(table2, false), grantUserName, false, false);
-    checkUserAclEntry(helper.getTableRootPaths(table1, false), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getGlobalRootPaths(), grantUserName, false, false);
+    checkUserAclEntry(FS, helper.getTableRootPaths(table2, false), grantUserName, false, false);
+    checkUserAclEntry(FS, helper.getTableRootPaths(table1, false), grantUserName, true, true);
   }
 
   @Test
@@ -253,8 +243,8 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table1 = TableName.valueOf(namespace, "t1");
-    TableName table2 = TableName.valueOf(namespace, "t2");
+    TableName table1 = TableName.valueOf(namespace, name.getMethodName() + ".1");
+    TableName table2 = TableName.valueOf(namespace, name.getMethodName() + ".2");
     String snapshot1 = namespace + "s1";
     String snapshot2 = namespace + "s2";
 
@@ -272,12 +262,12 @@ public class TestSnapshotScannerHDFSAclController {
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, unGrantUser, snapshot1, -1);
     assertTrue(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace));
     assertFalse(hasUserTableHdfsAcl(aclTable, grantUserName, table1));
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName, true, true);
     // grant N(W)
     SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, WRITE);
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, -1);
     assertFalse(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace));
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName, false, false);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName, false, false);
   }
 
   @Test
@@ -285,7 +275,7 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table1 = TableName.valueOf(namespace, "t1");
+    TableName table1 = TableName.valueOf(namespace, name.getMethodName());
     String snapshot1 = namespace + "s1";
 
     // create table1 and snapshot
@@ -301,9 +291,9 @@ public class TestSnapshotScannerHDFSAclController {
     // check scan snapshot
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, 6);
     assertFalse(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace));
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName, true, false);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName, true, false);
     assertTrue(hasUserTableHdfsAcl(aclTable, grantUserName, table1));
-    checkUserAclEntry(helper.getTableRootPaths(table1, false), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getTableRootPaths(table1, false), grantUserName, true, true);
   }
 
   @Test
@@ -311,7 +301,7 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
+    TableName table = TableName.valueOf(namespace, name.getMethodName());
     String snapshot = namespace + "t1";
 
     // create table1 and snapshot
@@ -326,9 +316,9 @@ public class TestSnapshotScannerHDFSAclController {
     // check scan snapshot
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6);
     assertFalse(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace));
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName, true, true);
     assertTrue(hasUserGlobalHdfsAcl(aclTable, grantUserName));
-    checkUserAclEntry(helper.getGlobalRootPaths(), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getGlobalRootPaths(), grantUserName, true, true);
   }
 
   @Test
@@ -337,14 +327,16 @@ public class TestSnapshotScannerHDFSAclController {
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
 
     String namespace = name.getMethodName();
-    TableName table1 = TableName.valueOf(namespace, "t1");
+    TableName table1 = TableName.valueOf(namespace, name.getMethodName());
     String snapshot1 = namespace + "s1";
     String snapshot2 = namespace + "s2";
 
+    LOG.info("Create table");
     try (Table t = TestHDFSAclHelper.createTable(TEST_UTIL, table1)) {
       TestHDFSAclHelper.put(t);
       admin.snapshot(snapshot1, table1);
       // table owner can scan table snapshot
+      LOG.info("Scan snapshot");
       TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL,
         User.createUserForTesting(conf, "owner", new String[] {}), snapshot1, 6);
       // grant table1 family(R)
@@ -360,7 +352,7 @@ public class TestSnapshotScannerHDFSAclController {
       TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, 6);
       TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot2, 10);
       assertTrue(hasUserTableHdfsAcl(aclTable, grantUserName, table1));
-      checkUserAclEntry(helper.getTableRootPaths(table1, false), grantUserName, true, true);
+      checkUserAclEntry(FS, helper.getTableRootPaths(table1, false), grantUserName, true, true);
     }
 
     // grant table1(W) with merging existing permissions
@@ -368,13 +360,13 @@ public class TestSnapshotScannerHDFSAclController {
       new UserPermission(grantUserName, Permission.newBuilder(table1).withActions(WRITE).build()),
       true);
     assertTrue(hasUserTableHdfsAcl(aclTable, grantUserName, table1));
-    checkUserAclEntry(helper.getTableRootPaths(table1, false), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getTableRootPaths(table1, false), grantUserName, true, true);
 
     // grant table1(W) without merging existing permissions
     TestHDFSAclHelper.grantOnTable(TEST_UTIL, grantUserName, table1, WRITE);
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, -1);
     assertFalse(hasUserTableHdfsAcl(aclTable, grantUserName, table1));
-    checkUserAclEntry(helper.getTableRootPaths(table1, false), grantUserName, false, false);
+    checkUserAclEntry(FS, helper.getTableRootPaths(table1, false), grantUserName, false, false);
   }
 
   @Test
@@ -382,7 +374,7 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
+    TableName table = TableName.valueOf(namespace, name.getMethodName());
     String snapshot = namespace + "s1";
 
     try (Table t = TestHDFSAclHelper.createMobTable(TEST_UTIL, table)) {
@@ -391,7 +383,7 @@ public class TestSnapshotScannerHDFSAclController {
       TestHDFSAclHelper.grantOnTable(TEST_UTIL, grantUserName, table, READ);
       TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6);
       assertTrue(hasUserTableHdfsAcl(aclTable, grantUserName, table));
-      checkUserAclEntry(helper.getTableRootPaths(table, false), grantUserName, true, true);
+      checkUserAclEntry(FS, helper.getTableRootPaths(table, false), grantUserName, true, true);
     }
   }
 
@@ -400,7 +392,7 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table1 = TableName.valueOf(namespace, "t1");
+    TableName table1 = TableName.valueOf(namespace, name.getMethodName());
     String snapshot1 = namespace + "t1";
 
     TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table1);
@@ -409,7 +401,7 @@ public class TestSnapshotScannerHDFSAclController {
     SecureTestUtil.revokeGlobal(TEST_UTIL, grantUserName, READ);
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, -1);
     assertFalse(hasUserGlobalHdfsAcl(aclTable, grantUserName));
-    checkUserAclEntry(helper.getGlobalRootPaths(), grantUserName, false, false);
+    checkUserAclEntry(FS, helper.getGlobalRootPaths(), grantUserName, false, false);
   }
 
   @Test
@@ -418,8 +410,8 @@ public class TestSnapshotScannerHDFSAclController {
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
 
     String namespace = name.getMethodName();
-    TableName table1 = TableName.valueOf(namespace, "t1");
     String snapshot1 = namespace + "s1";
+    TableName table1 = TableName.valueOf(namespace, name.getMethodName());
     TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table1);
     admin.snapshot(snapshot1, table1);
 
@@ -431,9 +423,9 @@ public class TestSnapshotScannerHDFSAclController {
     // check scan snapshot
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, 6);
     assertFalse(hasUserGlobalHdfsAcl(aclTable, grantUserName));
-    checkUserAclEntry(helper.getGlobalRootPaths(), grantUserName, false, false);
+    checkUserAclEntry(FS, helper.getGlobalRootPaths(), grantUserName, false, false);
     assertTrue(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace));
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName, true, true);
   }
 
   @Test
@@ -442,7 +434,7 @@ public class TestSnapshotScannerHDFSAclController {
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
 
     String namespace = name.getMethodName();
-    TableName table1 = TableName.valueOf(namespace, "t1");
+    TableName table1 = TableName.valueOf(namespace, name.getMethodName());
     String snapshot1 = namespace + "t1";
     TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table1);
     admin.snapshot(snapshot1, table1);
@@ -454,11 +446,11 @@ public class TestSnapshotScannerHDFSAclController {
     // check scan snapshot
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, 6);
     assertFalse(hasUserGlobalHdfsAcl(aclTable, grantUserName));
-    checkUserAclEntry(helper.getGlobalRootPaths(), grantUserName, false, false);
+    checkUserAclEntry(FS, helper.getGlobalRootPaths(), grantUserName, false, false);
     assertFalse(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace));
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName, true, false);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName, true, false);
     assertTrue(hasUserTableHdfsAcl(aclTable, grantUserName, table1));
-    checkUserAclEntry(helper.getTableRootPaths(table1, false), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getTableRootPaths(table1, false), grantUserName, true, true);
   }
 
   @Test
@@ -466,7 +458,7 @@ public class TestSnapshotScannerHDFSAclController {
     String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table1 = TableName.valueOf(namespace, "t1");
+    TableName table1 = TableName.valueOf(namespace, name.getMethodName());
     String snapshot1 = namespace + "s1";
     TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table1);
     admin.snapshot(snapshot1, table1);
@@ -477,7 +469,7 @@ public class TestSnapshotScannerHDFSAclController {
     // check scan snapshot
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, -1);
     assertFalse(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace));
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName, false, false);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName, false, false);
 
     // grant N(R), grant G(R) -> revoke N(R)
     SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ);
@@ -486,7 +478,7 @@ public class TestSnapshotScannerHDFSAclController {
     // check scan snapshot
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, 6);
     assertFalse(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace));
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName, true, true);
   }
 
   @Test
@@ -494,7 +486,7 @@ public class TestSnapshotScannerHDFSAclController {
     String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
+    TableName table = TableName.valueOf(namespace, name.getMethodName());
     String snapshot = namespace + "s1";
     TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table);
     admin.snapshot(snapshot, table);
@@ -506,9 +498,10 @@ public class TestSnapshotScannerHDFSAclController {
     // check scan snapshot
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6);
     assertFalse(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace));
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName, true, false);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName, true, false);
     assertTrue(hasUserTableHdfsAcl(aclTable, grantUserName, table));
-    checkUserAclEntry(helper.getTableRootPaths(table, false), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getTableRootPaths(table, false),
+      grantUserName, true, true);
   }
 
   @Test
@@ -516,7 +509,7 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
+    TableName table = TableName.valueOf(namespace, name.getMethodName());
     String snapshot = namespace + "t1";
     TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table);
     admin.snapshot(snapshot, table);
@@ -532,7 +525,7 @@ public class TestSnapshotScannerHDFSAclController {
     admin.revoke(new UserPermission(grantUserName, Permission.newBuilder(table).build()));
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, -1);
     assertFalse(hasUserTableHdfsAcl(aclTable, grantUserName, table));
-    checkUserAclEntry(helper.getTableRootPaths(table, false), grantUserName, false, false);
+    checkUserAclEntry(FS, helper.getTableRootPaths(table, false), grantUserName, false, false);
   }
 
   @Test
@@ -540,7 +533,7 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
+    TableName table = TableName.valueOf(namespace, name.getMethodName());
     String snapshot = namespace + "t1";
     TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table);
     admin.snapshot(snapshot, table);
@@ -551,9 +544,9 @@ public class TestSnapshotScannerHDFSAclController {
     admin.revoke(new UserPermission(grantUserName, Permission.newBuilder(table).build()));
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6);
     assertFalse(hasUserTableHdfsAcl(aclTable, grantUserName, table));
-    checkUserAclEntry(helper.getTableRootPaths(table, false), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getTableRootPaths(table, false), grantUserName, true, true);
     assertTrue(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace));
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName, true, true);
   }
 
   @Test
@@ -561,7 +554,7 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
+    TableName table = TableName.valueOf(namespace, name.getMethodName());
     String snapshot = namespace + "t1";
     TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table);
     admin.snapshot(snapshot, table);
@@ -572,9 +565,9 @@ public class TestSnapshotScannerHDFSAclController {
     admin.revoke(new UserPermission(grantUserName, Permission.newBuilder(table).build()));
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6);
     assertFalse(hasUserTableHdfsAcl(aclTable, grantUserName, table));
-    checkUserAclEntry(helper.getTableRootPaths(table, false), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getTableRootPaths(table, false), grantUserName, true, true);
     assertTrue(hasUserGlobalHdfsAcl(aclTable, grantUserName));
-    checkUserAclEntry(helper.getGlobalRootPaths(), grantUserName, true, true);
+    checkUserAclEntry(FS, helper.getGlobalRootPaths(), grantUserName, true, true);
   }
 
   @Test
@@ -585,7 +578,7 @@ public class TestSnapshotScannerHDFSAclController {
     User grantUser2 = User.createUserForTesting(conf, grantUserName2, new String[] {});
 
     String namespace = name.getMethodName();
-    TableName tableName = TableName.valueOf(namespace, "t1");
+    TableName tableName = TableName.valueOf(namespace, name.getMethodName());
     String snapshot = namespace + "s1";
     String snapshot2 = namespace + "s2";
     try (Table t = TestHDFSAclHelper.createTable(TEST_UTIL, tableName)) {
@@ -608,60 +601,10 @@ public class TestSnapshotScannerHDFSAclController {
       TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot2, 9);
       TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser2, snapshot2, 9);
       assertTrue(hasUserNamespaceHdfsAcl(aclTable, grantUserName2, namespace));
-      checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName2, true, true);
+      checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName2, true, true);
       assertTrue(hasUserTableHdfsAcl(aclTable, grantUserName, tableName));
-      checkUserAclEntry(helper.getTableRootPaths(tableName, false), grantUserName, true, true);
-      checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName, true, false);
-    }
-  }
-
-  @Test
-  public void testRestoreSnapshot() throws Exception {
-    final String grantUserName = name.getMethodName();
-    User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
-    String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
-    String snapshot = namespace + "s1";
-    String snapshot2 = namespace + "s2";
-    String snapshot3 = namespace + "s3";
-    TEST_UTIL.waitTableAvailable(PermissionStorage.ACL_TABLE_NAME);
-
-    try (Table t = TestHDFSAclHelper.createTable(TEST_UTIL, table)) {
-      TestHDFSAclHelper.put(t);
-      // grant t1, snapshot
-      TestHDFSAclHelper.grantOnTable(TEST_UTIL, grantUserName, table, READ);
-      admin.snapshot(snapshot, table);
-      // delete
-      admin.disableTable(table);
-      admin.deleteTable(table);
-      LOG.info("Before scan of shapshot!");
-      TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, -1);
-
-      // restore snapshot and restore acl
-      admin.restoreSnapshot(snapshot, true, true);
-      TestHDFSAclHelper.put2(t);
-      // snapshot
-      admin.snapshot(snapshot2, table);
-      TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6);
-      TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot2, 10);
-      assertTrue(hasUserTableHdfsAcl(aclTable, grantUserName, table));
-      checkUserAclEntry(helper.getTableRootPaths(table, false), grantUserName, true, true);
-
-      // delete
-      admin.disableTable(table);
-      admin.deleteTable(table);
-      // restore snapshot and skip restore acl
-      admin.restoreSnapshot(snapshot);
-      admin.snapshot(snapshot3, table);
-
-      LOG.info("CHECK");
-      TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, -1);
-      TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot2, -1);
-      TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot3, -1);
-      assertFalse(hasUserTableHdfsAcl(aclTable, grantUserName, table));
-      checkUserAclEntry(helper.getPathHelper().getDataTableDir(table), grantUserName, false, false);
-      checkUserAclEntry(helper.getPathHelper().getArchiveTableDir(table), grantUserName, true,
-        false);
+      checkUserAclEntry(FS, helper.getTableRootPaths(tableName, false), grantUserName, true, true);
+      checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName, true, false);
     }
   }
 
@@ -672,7 +615,7 @@ public class TestSnapshotScannerHDFSAclController {
     String grantUserName2 = namespace + "2";
     User grantUser1 = User.createUserForTesting(conf, grantUserName1, new String[] {});
     User grantUser2 = User.createUserForTesting(conf, grantUserName2, new String[] {});
-    TableName table = TableName.valueOf(namespace, "t1");
+    TableName table = TableName.valueOf(namespace, name.getMethodName());
     String snapshot1 = namespace + "t1";
 
     TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table);
@@ -688,16 +631,18 @@ public class TestSnapshotScannerHDFSAclController {
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser1, snapshot1, -1);
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser2, snapshot1, 6);
     assertTrue(hasUserNamespaceHdfsAcl(aclTable, grantUserName2, namespace));
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace), grantUserName2, true, true);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), grantUserName2, true, true);
     assertFalse(hasUserTableHdfsAcl(aclTable, grantUserName1, table));
-    checkUserAclEntry(helper.getPathHelper().getDataTableDir(table), grantUserName1, false, false);
-    checkUserAclEntry(helper.getPathHelper().getMobTableDir(table), grantUserName1, false, false);
-    checkUserAclEntry(helper.getPathHelper().getArchiveTableDir(table), grantUserName1, true,
+    checkUserAclEntry(FS, helper.getPathHelper().getDataTableDir(table), grantUserName1, false,
+      false);
+    checkUserAclEntry(FS, helper.getPathHelper().getMobTableDir(table), grantUserName1, false,
+      false);
+    checkUserAclEntry(FS, helper.getPathHelper().getArchiveTableDir(table), grantUserName1, true,
       false);
 
     // check tmp table directory does not exist
     Path tmpTableDir = helper.getPathHelper().getTmpTableDir(table);
-    assertFalse(fs.exists(tmpTableDir));
+    assertFalse(FS.exists(tmpTableDir));
   }
 
   @Test
@@ -705,7 +650,7 @@ public class TestSnapshotScannerHDFSAclController {
     String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
+    TableName table = TableName.valueOf(namespace, name.getMethodName());
     String snapshot = namespace + "t1";
     TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table);
     // snapshot
@@ -719,13 +664,13 @@ public class TestSnapshotScannerHDFSAclController {
     admin.deleteNamespace(namespace);
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6);
     assertFalse(hasUserNamespaceHdfsAcl(aclTable, grantUserName, namespace));
-    checkUserAclEntry(helper.getPathHelper().getArchiveNsDir(namespace), grantUserName, true,
+    checkUserAclEntry(FS, helper.getPathHelper().getArchiveNsDir(namespace), grantUserName, true,
       false);
 
     // check tmp namespace dir does not exist
-    assertFalse(fs.exists(helper.getPathHelper().getTmpNsDir(namespace)));
-    assertFalse(fs.exists(helper.getPathHelper().getDataNsDir(namespace)));
-    // assertFalse(fs.exists(helper.getPathHelper().getMobDataNsDir(namespace)));
+    assertFalse(FS.exists(helper.getPathHelper().getTmpNsDir(namespace)));
+    assertFalse(FS.exists(helper.getPathHelper().getDataNsDir(namespace)));
+    // assertFalse(fs.exists(FS, helper.getPathHelper().getMobDataNsDir(namespace)));
   }
 
   @Test
@@ -733,7 +678,7 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
+    TableName table = TableName.valueOf(namespace, name.getMethodName());
     String snapshot = namespace + "t1";
 
     TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table);
@@ -745,8 +690,8 @@ public class TestSnapshotScannerHDFSAclController {
     HFileCleaner cleaner = TEST_UTIL.getHBaseCluster().getMaster().getHFileCleaner();
     cleaner.choreForTesting();
     Path archiveTableDir = HFileArchiveUtil.getTableArchivePath(rootDir, table);
-    assertTrue(fs.exists(archiveTableDir));
-    checkUserAclEntry(helper.getTableRootPaths(table, false), grantUserName, true, true);
+    assertTrue(FS.exists(archiveTableDir));
+    checkUserAclEntry(FS, helper.getTableRootPaths(table, false), grantUserName, true, true);
 
     // Check SnapshotScannerHDFSAclCleaner method
     assertTrue(SnapshotScannerHDFSAclCleaner.isArchiveTableDir(archiveTableDir));
@@ -760,7 +705,7 @@ public class TestSnapshotScannerHDFSAclController {
   @Test
   public void testModifyTable1() throws Exception {
     String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
+    TableName table = TableName.valueOf(namespace, name.getMethodName());
     String snapshot = namespace + "t1";
 
     String tableUserName = name.getMethodName();
@@ -807,23 +752,23 @@ public class TestSnapshotScannerHDFSAclController {
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, globalUser, snapshot, 6);
     // check acl table storage and ACLs in dirs
     assertTrue(hasUserGlobalHdfsAcl(aclTable, globalUserName));
-    checkUserAclEntry(helper.getGlobalRootPaths(), globalUserName, true, true);
+    checkUserAclEntry(FS, helper.getGlobalRootPaths(), globalUserName, true, true);
     assertTrue(hasUserNamespaceHdfsAcl(aclTable, nsUserName, namespace));
-    checkUserAclEntry(helper.getNamespaceRootPaths(namespace), nsUserName, true, true);
+    checkUserAclEntry(FS, helper.getNamespaceRootPaths(namespace), nsUserName, true, true);
     assertTrue(hasUserTableHdfsAcl(aclTable, tableUserName, table));
-    checkUserAclEntry(helper.getTableRootPaths(table, false), tableUserName, true, true);
+    checkUserAclEntry(FS, helper.getTableRootPaths(table, false), tableUserName, true, true);
     for (String user : new String[] { tableUserName2, tableUserName3 }) {
       assertFalse(hasUserTableHdfsAcl(aclTable, user, table));
-      checkUserAclEntry(helper.getTableRootPaths(table, false), user, false, false);
+      checkUserAclEntry(FS, helper.getTableRootPaths(table, false), user, false, false);
     }
   }
 
   @Test
   public void testModifyTable2() throws Exception {
     String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
+    TableName table = TableName.valueOf(namespace, name.getMethodName() + ".1");
     String snapshot = namespace + "t1";
-    TableName table2 = TableName.valueOf(namespace, "t2");
+    TableName table2 = TableName.valueOf(namespace, name.getMethodName() + ".2");
 
     String tableUserName = name.getMethodName();
     User tableUser = User.createUserForTesting(conf, tableUserName, new String[] {});
@@ -864,16 +809,16 @@ public class TestSnapshotScannerHDFSAclController {
       tableUserName2, tableUserName3 };
     for (Path path : helper.getTableRootPaths(table, false)) {
       for (String user : users) {
-        checkUserAclEntry(path, user, false, false);
+        checkUserAclEntry(FS, path, user, false, false);
       }
     }
     String[] nsUsers = new String[] { globalUserName, globalUserName2, nsUserName };
     for (Path path : helper.getNamespaceRootPaths(namespace)) {
-      checkUserAclEntry(path, tableUserName, false, false);
-      checkUserAclEntry(path, tableUserName2, true, true);
-      checkUserAclEntry(path, tableUserName3, true, false);
+      checkUserAclEntry(FS, path, tableUserName, false, false);
+      checkUserAclEntry(FS, path, tableUserName2, true, true);
+      checkUserAclEntry(FS, path, tableUserName3, true, false);
       for (String user : nsUsers) {
-        checkUserAclEntry(path, user, true, true);
+        checkUserAclEntry(FS, path, user, true, true);
       }
     }
     assertTrue(hasUserNamespaceHdfsAcl(aclTable, nsUserName, namespace));
@@ -886,8 +831,8 @@ public class TestSnapshotScannerHDFSAclController {
     final String grantUserName = name.getMethodName();
     User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
     String namespace = name.getMethodName();
-    TableName table = TableName.valueOf(namespace, "t1");
-    TableName table2 = TableName.valueOf(namespace, "t2");
+    TableName table = TableName.valueOf(namespace, name.getMethodName() + ".1");
+    TableName table2 = TableName.valueOf(namespace, name.getMethodName() + ".2");
     String snapshot = namespace + "t1";
     admin.createNamespace(NamespaceDescriptor.create(namespace).build());
 
@@ -896,14 +841,14 @@ public class TestSnapshotScannerHDFSAclController {
     // make some region files in tmp dir and check if master archive these region correctly
     Path tmpTableDir = helper.getPathHelper().getTmpTableDir(table2);
     // make a empty region dir, this is an error region
-    fs.mkdirs(new Path(tmpTableDir, "1"));
+    FS.mkdirs(new Path(tmpTableDir, "1"));
     // copy regions from data dir, this is a valid region
-    for (Path regionDir : FSUtils.getRegionDirs(fs,
+    for (Path regionDir : FSUtils.getRegionDirs(FS,
       helper.getPathHelper().getDataTableDir(table2))) {
-      FSUtils.copyFilesParallel(fs, regionDir, fs,
+      FSUtils.copyFilesParallel(FS, regionDir, FS,
         new Path(tmpTableDir, regionDir.getName() + "abc"), conf, 1);
     }
-    assertEquals(4, fs.listStatus(tmpTableDir).length);
+    assertEquals(4, FS.listStatus(tmpTableDir).length);
 
     // grant N(R)
     SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ);
@@ -918,9 +863,9 @@ public class TestSnapshotScannerHDFSAclController {
     helper = new SnapshotScannerHDFSAclHelper(conf, admin.getConnection());
 
     Path tmpNsDir = helper.getPathHelper().getTmpNsDir(namespace);
-    assertTrue(fs.exists(tmpNsDir));
+    assertTrue(FS.exists(tmpNsDir));
     // check all regions in tmp table2 dir are archived
-    assertEquals(0, fs.listStatus(tmpTableDir).length);
+    assertEquals(0, FS.listStatus(tmpTableDir).length);
 
     // create table1 and snapshot
     TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table);
@@ -929,14 +874,14 @@ public class TestSnapshotScannerHDFSAclController {
     TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6);
   }
 
-  private void checkUserAclEntry(List<Path> paths, String user, boolean requireAccessAcl,
-      boolean requireDefaultAcl) throws Exception {
+  static void checkUserAclEntry(FileSystem fs, List<Path> paths, String user,
+      boolean requireAccessAcl, boolean requireDefaultAcl) throws Exception {
     for (Path path : paths) {
-      checkUserAclEntry(path, user, requireAccessAcl, requireDefaultAcl);
+      checkUserAclEntry(fs, path, user, requireAccessAcl, requireDefaultAcl);
     }
   }
 
-  private void checkUserAclEntry(Path path, String userName, boolean requireAccessAcl,
+  static void checkUserAclEntry(FileSystem fs, Path path, String userName, boolean requireAccessAcl,
       boolean requireDefaultAcl) throws IOException {
     boolean accessAclEntry = false;
     boolean defaultAclEntry = false;
@@ -957,139 +902,3 @@ public class TestSnapshotScannerHDFSAclController {
     assertEquals(message, requireDefaultAcl, defaultAclEntry);
   }
 }
-
-final class TestHDFSAclHelper {
-  private static final Logger LOG = LoggerFactory.getLogger(TestHDFSAclHelper.class);
-
-  private TestHDFSAclHelper() {
-  }
-
-  static void grantOnTable(HBaseTestingUtility util, String user, TableName tableName,
-      Permission.Action... actions) throws Exception {
-    SecureTestUtil.grantOnTable(util, user, tableName, null, null, actions);
-  }
-
-  static void createNamespace(HBaseTestingUtility util, String namespace) throws IOException {
-    if (Arrays.stream(util.getAdmin().listNamespaceDescriptors())
-        .noneMatch(ns -> ns.getName().equals(namespace))) {
-      NamespaceDescriptor namespaceDescriptor = NamespaceDescriptor.create(namespace).build();
-      util.getAdmin().createNamespace(namespaceDescriptor);
-    }
-  }
-
-  static Table createTable(HBaseTestingUtility util, TableName tableName) throws IOException {
-    createNamespace(util, tableName.getNamespaceAsString());
-    TableDescriptor td = getTableDescriptorBuilder(util, tableName)
-        .setValue(SnapshotScannerHDFSAclHelper.ACL_SYNC_TO_HDFS_ENABLE, "true").build();
-    byte[][] splits = new byte[][] { Bytes.toBytes("2"), Bytes.toBytes("4") };
-    return util.createTable(td, splits);
-  }
-
-  static Table createMobTable(HBaseTestingUtility util, TableName tableName) throws IOException {
-    createNamespace(util, tableName.getNamespaceAsString());
-    TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName)
-        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN1).setMobEnabled(true)
-            .setMobThreshold(0).build())
-        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN2).setMobEnabled(true)
-            .setMobThreshold(0).build())
-        .setOwner(User.createUserForTesting(util.getConfiguration(), "owner", new String[] {}))
-        .setValue(SnapshotScannerHDFSAclHelper.ACL_SYNC_TO_HDFS_ENABLE, "true").build();
-    byte[][] splits = new byte[][] { Bytes.toBytes("2"), Bytes.toBytes("4") };
-    return util.createTable(td, splits);
-  }
-
-  static TableDescriptor createUserScanSnapshotDisabledTable(HBaseTestingUtility util,
-      TableName tableName) throws IOException {
-    createNamespace(util, tableName.getNamespaceAsString());
-    TableDescriptor td = getTableDescriptorBuilder(util, tableName).build();
-    byte[][] splits = new byte[][] { Bytes.toBytes("2"), Bytes.toBytes("4") };
-    try (Table t = util.createTable(td, splits)) {
-      put(t);
-    }
-    return td;
-  }
-
-  private static TableDescriptorBuilder getTableDescriptorBuilder(HBaseTestingUtility util,
-      TableName tableName) {
-    return TableDescriptorBuilder.newBuilder(tableName)
-        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN1).build())
-        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN2).build())
-        .setOwner(User.createUserForTesting(util.getConfiguration(), "owner", new String[] {}));
-  }
-
-  static void createTableAndPut(HBaseTestingUtility util, TableName tableNam) throws IOException {
-    try (Table t = createTable(util, tableNam)) {
-      put(t);
-    }
-  }
-
-  static final byte[] COLUMN1 = Bytes.toBytes("A");
-  static final byte[] COLUMN2 = Bytes.toBytes("B");
-
-  static void put(Table hTable) throws IOException {
-    List<Put> puts = new ArrayList<>();
-    for (int i = 0; i < 6; i++) {
-      Put put = new Put(Bytes.toBytes(i));
-      put.addColumn(COLUMN1, null, Bytes.toBytes(i));
-      put.addColumn(COLUMN2, null, Bytes.toBytes(i + 1));
-      puts.add(put);
-    }
-    hTable.put(puts);
-  }
-
-  static void put2(Table hTable) throws IOException {
-    List<Put> puts = new ArrayList<>();
-    for (int i = 0; i < 10; i++) {
-      if (i == 5) {
-        continue;
-      }
-      Put put = new Put(Bytes.toBytes(i));
-      put.addColumn(COLUMN1, null, Bytes.toBytes(i + 2));
-      put.addColumn(COLUMN2, null, Bytes.toBytes(i + 3));
-      puts.add(put);
-    }
-    hTable.put(puts);
-  }
-
-  /**
-   * Check if user is able to read expected rows from the specific snapshot
-   * @param user the specific user
-   * @param snapshot the snapshot to be scanned
-   * @param expectedRowCount expected row count read from snapshot, -1 if expects
-   *          AccessControlException
-   * @throws IOException user scan snapshot error
-   * @throws InterruptedException user scan snapshot error
-   */
-  static void canUserScanSnapshot(HBaseTestingUtility util, User user, String snapshot,
-      int expectedRowCount) throws IOException, InterruptedException {
-    PrivilegedExceptionAction<Void> action =
-        getScanSnapshotAction(util.getConfiguration(), snapshot, expectedRowCount);
-    user.runAs(action);
-  }
-
-  private static PrivilegedExceptionAction<Void> getScanSnapshotAction(Configuration conf,
-      String snapshotName, long expectedRowCount) {
-    return () -> {
-      try {
-        Path restoreDir = new Path(SnapshotScannerHDFSAclHelper.SNAPSHOT_RESTORE_TMP_DIR_DEFAULT);
-        Scan scan = new Scan();
-        TableSnapshotScanner scanner =
-            new TableSnapshotScanner(conf, restoreDir, snapshotName, scan);
-        int rowCount = 0;
-        while (true) {
-          Result result = scanner.next();
-          if (result == null) {
-            break;
-          }
-          rowCount++;
-        }
-        scanner.close();
-        assertEquals(expectedRowCount, rowCount);
-      } catch (Exception e) {
-        LOG.debug("Scan snapshot error, snapshot {}", snapshotName, e);
-        assertEquals(expectedRowCount, -1);
-      }
-      return null;
-    };
-  }
-}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController2.java
new file mode 100644
index 0000000..da6ac7e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController2.java
@@ -0,0 +1,185 @@
+/*
+ * 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.security.access;
+
+import static org.apache.hadoop.hbase.security.access.Permission.Action.READ;
+import static org.apache.hadoop.hbase.security.access.SnapshotScannerHDFSAclController.SnapshotScannerHDFSAclStorage.hasUserTableHdfsAcl;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Separated from {@link TestSnapshotScannerHDFSAclController}. Uses facility from that class.
+ * @see TestSnapshotScannerHDFSAclController
+ */
+@Category({ SecurityTests.class, LargeTests.class })
+public class TestSnapshotScannerHDFSAclController2 {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestSnapshotScannerHDFSAclController2.class);
+  @Rule
+  public TestName name = new TestName();
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestSnapshotScannerHDFSAclController2.class);
+
+  private static final String UN_GRANT_USER = "un_grant_user";
+  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf = TEST_UTIL.getConfiguration();
+  private static Admin admin = null;
+  private static SnapshotScannerHDFSAclHelper helper;
+  private static Table aclTable;
+  private static FileSystem FS;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    // enable hdfs acl and set umask to 027
+    conf.setBoolean("dfs.namenode.acls.enabled", true);
+    conf.set("fs.permissions.umask-mode", "027");
+    // enable hbase hdfs acl feature
+    conf.setBoolean(SnapshotScannerHDFSAclHelper.ACL_SYNC_TO_HDFS_ENABLE, true);
+    // enable secure
+    conf.set(User.HBASE_SECURITY_CONF_KEY, "simple");
+    conf.set(SnapshotScannerHDFSAclHelper.SNAPSHOT_RESTORE_TMP_DIR,
+      SnapshotScannerHDFSAclHelper.SNAPSHOT_RESTORE_TMP_DIR_DEFAULT);
+    SecureTestUtil.enableSecurity(conf);
+    // add SnapshotScannerHDFSAclController coprocessor
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+      conf.get(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY) + ","
+          + SnapshotScannerHDFSAclController.class.getName());
+
+    TEST_UTIL.startMiniCluster();
+    TEST_UTIL.waitTableAvailable(PermissionStorage.ACL_TABLE_NAME);
+    admin = TEST_UTIL.getAdmin();
+    Path rootDir = TEST_UTIL.getDefaultRootDirPath();
+    FS = rootDir.getFileSystem(conf);
+    User unGrantUser = User.createUserForTesting(conf, UN_GRANT_USER, new String[] {});
+    helper = new SnapshotScannerHDFSAclHelper(conf, admin.getConnection());
+
+    // set hbase directory permission
+    FsPermission commonDirectoryPermission =
+        new FsPermission(conf.get(SnapshotScannerHDFSAclHelper.COMMON_DIRECTORY_PERMISSION,
+          SnapshotScannerHDFSAclHelper.COMMON_DIRECTORY_PERMISSION_DEFAULT));
+    Path path = rootDir;
+    while (path != null) {
+      FS.setPermission(path, commonDirectoryPermission);
+      path = path.getParent();
+    }
+    // set restore directory permission
+    Path restoreDir = new Path(SnapshotScannerHDFSAclHelper.SNAPSHOT_RESTORE_TMP_DIR_DEFAULT);
+    if (!FS.exists(restoreDir)) {
+      FS.mkdirs(restoreDir);
+      FS.setPermission(restoreDir,
+        new FsPermission(
+            conf.get(SnapshotScannerHDFSAclHelper.SNAPSHOT_RESTORE_DIRECTORY_PERMISSION,
+              SnapshotScannerHDFSAclHelper.SNAPSHOT_RESTORE_DIRECTORY_PERMISSION_DEFAULT)));
+    }
+    path = restoreDir.getParent();
+    while (path != null) {
+      FS.setPermission(path, commonDirectoryPermission);
+      path = path.getParent();
+    }
+
+    SnapshotScannerHDFSAclController coprocessor = TEST_UTIL.getHBaseCluster().getMaster()
+        .getMasterCoprocessorHost().findCoprocessor(SnapshotScannerHDFSAclController.class);
+    TEST_UTIL.waitFor(1200000, () -> coprocessor.checkInitialized("check initialized"));
+    aclTable = admin.getConnection().getTable(PermissionStorage.ACL_TABLE_NAME);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testRestoreSnapshot() throws Exception {
+    final String grantUserName = name.getMethodName();
+    User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
+    String namespace = name.getMethodName();
+    TableName table = TableName.valueOf(namespace, name.getMethodName());
+    String snapshot = namespace + "s1";
+    String snapshot2 = namespace + "s2";
+    String snapshot3 = namespace + "s3";
+
+    LOG.info("Create {}", table);
+    try (Table t = TestHDFSAclHelper.createTable(TEST_UTIL, table)) {
+      TestHDFSAclHelper.put(t);
+      // grant t1, snapshot
+      LOG.info("Grant {}", table);
+      TestHDFSAclHelper.grantOnTable(TEST_UTIL, grantUserName, table, READ);
+      admin.snapshot(snapshot, table);
+      // delete
+      admin.disableTable(table);
+      admin.deleteTable(table);
+      LOG.info("Before scan of shapshot! {}", table);
+      TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, -1);
+
+      // restore snapshot and restore acl
+      admin.restoreSnapshot(snapshot, true, true);
+      TestHDFSAclHelper.put2(t);
+      // snapshot
+      admin.snapshot(snapshot2, table);
+      TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6);
+      TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot2, 10);
+      assertTrue(hasUserTableHdfsAcl(aclTable, grantUserName, table));
+      TestSnapshotScannerHDFSAclController.
+        checkUserAclEntry(FS, helper.getTableRootPaths(table, false),
+          grantUserName, true, true);
+
+      // delete
+      admin.disableTable(table);
+      admin.deleteTable(table);
+      // restore snapshot and skip restore acl
+      admin.restoreSnapshot(snapshot);
+      admin.snapshot(snapshot3, table);
+
+      LOG.info("CHECK");
+      TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, -1);
+      TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot2, -1);
+      TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot3, -1);
+      assertFalse(hasUserTableHdfsAcl(aclTable, grantUserName, table));
+      TestSnapshotScannerHDFSAclController.
+        checkUserAclEntry(FS, helper.getPathHelper().getDataTableDir(table),
+          grantUserName, false, false);
+      TestSnapshotScannerHDFSAclController.
+        checkUserAclEntry(FS, helper.getPathHelper().getArchiveTableDir(table),
+          grantUserName, true, false);
+    }
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
index 903c89e..5025861 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
@@ -185,7 +185,10 @@ public class TestZKSecretWatcher {
 
     // verify that the expired key has been removed
     Waiter.waitFor(TEST_UTIL.getConfiguration(), 30000,
-      () -> KEY_SLAVE.getKey(key1.getKeyId()) == null);
+      () -> {
+      AuthenticationKey k = KEY_SLAVE.getKey(key1.getKeyId());
+      LOG.info("AuthKey1={}", k);
+      return k == null;});
     assertNull("key1=" + KEY_SLAVE.getKey(key1.getKeyId()),
       KEY_SLAVE.getKey(key1.getKeyId()));