You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2020/07/30 07:34:11 UTC

[GitHub] [hadoop] smengcl opened a new pull request #2176: [WIP] HDFS-15492. Make trash root inside each snapshottable directory

smengcl opened a new pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176


   https://issues.apache.org/jira/browse/HDFS-15492
   
   # TODOs
   
   - [x] Create trash under snapshot
   - [ ] Make this a config and disable it by default
   - [ ] Add unit test
   - [ ] Check functionality when both encryption zone and snapshot is enabled on a directory
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] bshashikant commented on a change in pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#discussion_r466517871



##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
##########
@@ -2144,4 +2146,293 @@ public void testECCloseCommittedBlock() throws Exception {
       LambdaTestUtils.intercept(IOException.class, "", () -> str.close());
     }
   }
+
+  @Test
+  public void testGetTrashRoot() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+
+      Path trBeforeAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trBeforeAllowSnapshotStr = trBeforeAllowSnapshot.toUri().getPath();
+      // The trash root should be in user home directory
+      String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+      assertTrue(trBeforeAllowSnapshotStr.startsWith(homeDirStr));
+
+      dfs.allowSnapshot(testDir);
+
+      Path trAfterAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trAfterAllowSnapshotStr = trAfterAllowSnapshot.toUri().getPath();
+      // The trash root should now be in the snapshot root
+      String testDirStr = testDir.toUri().getPath();
+      assertTrue(trAfterAllowSnapshotStr.startsWith(testDirStr));
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private boolean isPathInUserHome(String pathStr, DistributedFileSystem dfs) {
+    String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+    return pathStr.startsWith(homeDirStr);
+  }
+
+  @Test
+  public void testGetTrashRoots() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+      // Create user trash
+      Path currUserHome = dfs.getHomeDirectory();
+      Path currUserTrash = new Path(currUserHome, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(currUserTrash);
+      // Create trash inside test directory
+      Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      Path testDirTrashCurrUser = new Path(testDirTrash,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+      dfs.mkdirs(testDirTrashCurrUser);
+
+      Collection<FileStatus> trashRoots = dfs.getTrashRoots(false);
+      // getTrashRoots should only return 1 empty user trash in the home dir now
+      assertEquals(1, trashRoots.size());
+      FileStatus firstFileStatus = trashRoots.iterator().next();
+      String pathStr = firstFileStatus.getPath().toUri().getPath();
+      assertTrue(isPathInUserHome(pathStr, dfs));
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRoots, trashRootsAllUsers);
+
+      dfs.allowSnapshot(testDir);
+
+      Collection<FileStatus> trashRootsAfter = dfs.getTrashRoots(false);
+      // getTrashRoots should return 1 more trash root inside snapshottable dir
+      assertEquals(trashRoots.size() + 1, trashRootsAfter.size());
+      boolean foundUserHomeTrash = false;
+      boolean foundSnapDirUserTrash = false;
+      String testDirStr = testDir.toUri().getPath();
+      for (FileStatus fileStatus : trashRootsAfter) {
+        String currPathStr = fileStatus.getPath().toUri().getPath();
+        if (isPathInUserHome(currPathStr, dfs)) {
+          foundUserHomeTrash = true;
+        } else if (currPathStr.startsWith(testDirStr)) {
+          foundSnapDirUserTrash = true;
+        }
+      }
+      assertTrue(foundUserHomeTrash);
+      assertTrue(foundSnapDirUserTrash);
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAfterAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter, trashRootsAfterAllUsers);
+
+      // Create trash root for user0
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user0");
+      String user0HomeStr = DFSUtilClient.getHomeDirectory(conf, ugi);
+      Path user0Trash = new Path(user0HomeStr, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(user0Trash);
+      // allUsers flag set to false should be unaffected
+      Collection<FileStatus> trashRootsAfter2 = dfs.getTrashRoots(false);
+      assertEquals(trashRootsAfter, trashRootsAfter2);
+      // allUsers flag set to true should include new user's trash
+      trashRootsAfter2 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter.size() + 1, trashRootsAfter2.size());
+
+      // Create trash root inside the snapshottable directory for user0
+      Path testDirTrashUser0 = new Path(testDirTrash, ugi.getShortUserName());
+      dfs.mkdirs(testDirTrashUser0);
+      Collection<FileStatus> trashRootsAfter3 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter2.size() + 1, trashRootsAfter3.size());
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootsOnSnapshottableDirWithEZ()
+      throws IOException, NoSuchAlgorithmException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    // Set encryption zone config
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    // Create key for EZ
+    final KeyProvider provider =
+        cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey("key", options);
+    provider.flush();
+
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test2/");
+      dfs.mkdirs(testDir);
+      dfs.createEncryptionZone(testDir, "key");
+
+      // Create trash inside test directory
+      Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      Path testDirTrashCurrUser = new Path(testDirTrash,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+      dfs.mkdirs(testDirTrashCurrUser);
+
+      Collection<FileStatus> trashRoots = dfs.getTrashRoots(false);
+      assertEquals(1, trashRoots.size());
+      FileStatus firstFileStatus = trashRoots.iterator().next();
+      String pathStr = firstFileStatus.getPath().toUri().getPath();
+      String testDirStr = testDir.toUri().getPath();
+      assertTrue(pathStr.startsWith(testDirStr));
+
+      dfs.allowSnapshot(testDir);
+
+      Collection<FileStatus> trashRootsAfter = dfs.getTrashRoots(false);
+      // getTrashRoots should give the same result
+      assertEquals(trashRoots, trashRootsAfter);
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootOnSnapshottableDirInEZ()
+      throws IOException, NoSuchAlgorithmException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    // Set EZ config
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    // Create key for EZ
+    final KeyProvider provider =
+        cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey("key", options);
+    provider.flush();
+
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+
+      Path testDir = new Path("/ssgtr/test3ez/");
+      dfs.mkdirs(testDir);
+      dfs.createEncryptionZone(testDir, "key");
+      Path testSubD = new Path(testDir, "sssubdir");
+      Path file1Path = new Path(testSubD, "file1");
+      dfs.create(file1Path);
+
+      final Path trBefore = dfs.getTrashRoot(file1Path);
+      final String trBeforeStr = trBefore.toUri().getPath();
+      // The trash root should be directly under testDir
+      final Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      final String testDirTrashStr = testDirTrash.toUri().getPath();
+      assertTrue(trBeforeStr.startsWith(testDirTrashStr));
+
+      dfs.allowSnapshot(testSubD);
+      final Path trAfter = dfs.getTrashRoot(file1Path);
+      final String trAfterStr = trAfter.toUri().getPath();
+      // The trash is now located in the dir inside
+      final Path testSubDirTrash = new Path(testSubD, FileSystem.TRASH_PREFIX);
+      final String testSubDirTrashStr = testSubDirTrash.toUri().getPath();
+      assertTrue(trAfterStr.startsWith(testSubDirTrashStr));

Review comment:
       can we change this to equals rather strtsWith??

##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
##########
@@ -2144,4 +2146,293 @@ public void testECCloseCommittedBlock() throws Exception {
       LambdaTestUtils.intercept(IOException.class, "", () -> str.close());
     }
   }
+
+  @Test
+  public void testGetTrashRoot() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+
+      Path trBeforeAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trBeforeAllowSnapshotStr = trBeforeAllowSnapshot.toUri().getPath();
+      // The trash root should be in user home directory
+      String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+      assertTrue(trBeforeAllowSnapshotStr.startsWith(homeDirStr));
+
+      dfs.allowSnapshot(testDir);
+
+      Path trAfterAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trAfterAllowSnapshotStr = trAfterAllowSnapshot.toUri().getPath();
+      // The trash root should now be in the snapshot root
+      String testDirStr = testDir.toUri().getPath();
+      assertTrue(trAfterAllowSnapshotStr.startsWith(testDirStr));
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private boolean isPathInUserHome(String pathStr, DistributedFileSystem dfs) {
+    String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+    return pathStr.startsWith(homeDirStr);
+  }
+
+  @Test
+  public void testGetTrashRoots() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+      // Create user trash
+      Path currUserHome = dfs.getHomeDirectory();
+      Path currUserTrash = new Path(currUserHome, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(currUserTrash);
+      // Create trash inside test directory
+      Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      Path testDirTrashCurrUser = new Path(testDirTrash,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+      dfs.mkdirs(testDirTrashCurrUser);
+
+      Collection<FileStatus> trashRoots = dfs.getTrashRoots(false);
+      // getTrashRoots should only return 1 empty user trash in the home dir now
+      assertEquals(1, trashRoots.size());
+      FileStatus firstFileStatus = trashRoots.iterator().next();
+      String pathStr = firstFileStatus.getPath().toUri().getPath();
+      assertTrue(isPathInUserHome(pathStr, dfs));
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRoots, trashRootsAllUsers);
+
+      dfs.allowSnapshot(testDir);
+
+      Collection<FileStatus> trashRootsAfter = dfs.getTrashRoots(false);
+      // getTrashRoots should return 1 more trash root inside snapshottable dir
+      assertEquals(trashRoots.size() + 1, trashRootsAfter.size());
+      boolean foundUserHomeTrash = false;
+      boolean foundSnapDirUserTrash = false;
+      String testDirStr = testDir.toUri().getPath();
+      for (FileStatus fileStatus : trashRootsAfter) {
+        String currPathStr = fileStatus.getPath().toUri().getPath();
+        if (isPathInUserHome(currPathStr, dfs)) {
+          foundUserHomeTrash = true;
+        } else if (currPathStr.startsWith(testDirStr)) {
+          foundSnapDirUserTrash = true;
+        }
+      }
+      assertTrue(foundUserHomeTrash);
+      assertTrue(foundSnapDirUserTrash);
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAfterAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter, trashRootsAfterAllUsers);
+
+      // Create trash root for user0
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user0");
+      String user0HomeStr = DFSUtilClient.getHomeDirectory(conf, ugi);
+      Path user0Trash = new Path(user0HomeStr, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(user0Trash);
+      // allUsers flag set to false should be unaffected
+      Collection<FileStatus> trashRootsAfter2 = dfs.getTrashRoots(false);
+      assertEquals(trashRootsAfter, trashRootsAfter2);
+      // allUsers flag set to true should include new user's trash
+      trashRootsAfter2 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter.size() + 1, trashRootsAfter2.size());
+
+      // Create trash root inside the snapshottable directory for user0
+      Path testDirTrashUser0 = new Path(testDirTrash, ugi.getShortUserName());
+      dfs.mkdirs(testDirTrashUser0);
+      Collection<FileStatus> trashRootsAfter3 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter2.size() + 1, trashRootsAfter3.size());
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootsOnSnapshottableDirWithEZ()
+      throws IOException, NoSuchAlgorithmException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    // Set encryption zone config
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    // Create key for EZ
+    final KeyProvider provider =
+        cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey("key", options);
+    provider.flush();
+
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test2/");
+      dfs.mkdirs(testDir);
+      dfs.createEncryptionZone(testDir, "key");
+
+      // Create trash inside test directory
+      Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      Path testDirTrashCurrUser = new Path(testDirTrash,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+      dfs.mkdirs(testDirTrashCurrUser);
+
+      Collection<FileStatus> trashRoots = dfs.getTrashRoots(false);
+      assertEquals(1, trashRoots.size());
+      FileStatus firstFileStatus = trashRoots.iterator().next();
+      String pathStr = firstFileStatus.getPath().toUri().getPath();
+      String testDirStr = testDir.toUri().getPath();
+      assertTrue(pathStr.startsWith(testDirStr));
+
+      dfs.allowSnapshot(testDir);
+
+      Collection<FileStatus> trashRootsAfter = dfs.getTrashRoots(false);
+      // getTrashRoots should give the same result
+      assertEquals(trashRoots, trashRootsAfter);
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootOnSnapshottableDirInEZ()
+      throws IOException, NoSuchAlgorithmException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    // Set EZ config
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    // Create key for EZ
+    final KeyProvider provider =
+        cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey("key", options);
+    provider.flush();
+
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+
+      Path testDir = new Path("/ssgtr/test3ez/");
+      dfs.mkdirs(testDir);
+      dfs.createEncryptionZone(testDir, "key");
+      Path testSubD = new Path(testDir, "sssubdir");
+      Path file1Path = new Path(testSubD, "file1");
+      dfs.create(file1Path);
+
+      final Path trBefore = dfs.getTrashRoot(file1Path);
+      final String trBeforeStr = trBefore.toUri().getPath();
+      // The trash root should be directly under testDir
+      final Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      final String testDirTrashStr = testDirTrash.toUri().getPath();
+      assertTrue(trBeforeStr.startsWith(testDirTrashStr));
+
+      dfs.allowSnapshot(testSubD);
+      final Path trAfter = dfs.getTrashRoot(file1Path);
+      final String trAfterStr = trAfter.toUri().getPath();
+      // The trash is now located in the dir inside
+      final Path testSubDirTrash = new Path(testSubD, FileSystem.TRASH_PREFIX);
+      final String testSubDirTrashStr = testSubDirTrash.toUri().getPath();
+      assertTrue(trAfterStr.startsWith(testSubDirTrashStr));

Review comment:
       can we change this to equals rather startsWith??




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] smengcl commented on a change in pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#discussion_r466236175



##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
##########
@@ -244,6 +244,10 @@
       "dfs.namenode.snapshot.capture.openfiles";
   boolean DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES_DEFAULT = false;
 
+  String DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED =
+      "dfs.namenode.snapshot.trashroot.enabled";
+  boolean DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED_DEFAULT = false;
+
   String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS =

Review comment:
       You are right. Done in 6b0a2585e837d7386fc52f58fc32e668d2440f5d.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] smengcl commented on a change in pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#discussion_r465975831



##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
##########
@@ -244,6 +244,10 @@
       "dfs.namenode.snapshot.capture.openfiles";
   boolean DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES_DEFAULT = false;
 
+  String DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED =
+      "dfs.namenode.snapshot.trashroot.enabled";
+  boolean DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED_DEFAULT = false;
+
   String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS =

Review comment:
       makes sense. will hide it from the client.
   
   but we do want to put this in hdfs-default.xml so people can learn to enable this on NN right?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] smengcl merged pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
smengcl merged pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-668586238


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 32s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  0s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  22m 34s |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 59s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  18m 33s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  checkstyle  |   3m  3s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   4m 11s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  22m 26s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 30s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   4m  7s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +0 :ok: |  spotbugs  |   3m 15s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   8m 16s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 53s |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 27s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | -1 :x: |  cc  |  20m 27s |  root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 39 new + 123 unchanged - 39 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  20m 27s |  the patch passed  |
   | +1 :green_heart: |  compile  |  18m 38s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | -1 :x: |  cc  |  18m 38s |  root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09 with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09 generated 21 new + 141 unchanged - 21 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  18m 38s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   3m 21s |  root: The patch generated 3 new + 826 unchanged - 0 fixed = 829 total (was 826)  |
   | +1 :green_heart: |  mvnsite  |   4m  3s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  16m 57s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 43s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   4m  7s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  findbugs  |   9m 27s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  10m 13s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 26s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  | 151m 58s |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  3s |  The patch does not generate ASF License warnings.  |
   |  |   | 354m 36s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFS |
   |   | hadoop.tools.TestHdfsConfigFields |
   |   | hadoop.hdfs.server.diskbalancer.TestDiskBalancerWithMockMover |
   |   | hadoop.hdfs.server.balancer.TestBalancerWithEncryptedTransfer |
   |   | hadoop.hdfs.server.balancer.TestBalancer |
   |   | hadoop.hdfs.server.balancer.TestBalancerWithHANameNodes |
   |   | hadoop.hdfs.server.balancer.TestBalancerRPCDelay |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   |   | hadoop.hdfs.server.namenode.TestNameNodeRetryCacheMetrics |
   |   | hadoop.hdfs.server.namenode.metrics.TestNameNodeMetrics |
   |   | hadoop.hdfs.server.blockmanagement.TestPendingDataNodeMessages |
   |   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
   |   | hadoop.hdfs.server.balancer.TestBalancerWithMultipleNameNodes |
   |   | hadoop.hdfs.server.balancer.TestBalancerWithNodeGroup |
   |   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   |   | hadoop.hdfs.server.balancer.TestBalancerWithSaslDataTransfer |
   |   | hadoop.hdfs.server.diskbalancer.command.TestDiskBalancerCommand |
   |   | hadoop.hdfs.server.datanode.TestBPOfferService |
   |   | hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2176 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 132b8352a85f 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / ab2b3df2de1 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/7/artifact/out/diff-compile-cc-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/7/artifact/out/diff-compile-cc-root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/7/artifact/out/diff-checkstyle-root.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/7/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/7/testReport/ |
   | Max. process+thread count | 3804 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/7/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] smengcl commented on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-670697255


   I rebased the commits on to the latest trunk. Also made `dfs.namenode.snapshot.trashroot.enabled` a private config in FSNameSystem.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-668867372


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 27s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  0s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  19m 51s |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 15s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  17m 20s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  checkstyle  |   2m 57s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   4m  2s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 30s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 42s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   3m 53s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +0 :ok: |  spotbugs  |   3m 14s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   8m  3s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 55s |  the patch passed  |
   | +1 :green_heart: |  compile  |  19m 57s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | -1 :x: |  cc  |  19m 57s |  root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 35 new + 127 unchanged - 35 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  19m 57s |  the patch passed  |
   | +1 :green_heart: |  compile  |  17m 58s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | -1 :x: |  cc  |  17m 58s |  root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09 with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09 generated 34 new + 128 unchanged - 34 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  17m 58s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 47s |  root: The patch generated 1 new + 826 unchanged - 0 fixed = 827 total (was 826)  |
   | +1 :green_heart: |  mvnsite  |   3m 59s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  13m 56s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 43s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   3m 53s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  findbugs  |   8m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 50s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 25s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  | 125m 40s |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  5s |  The patch does not generate ASF License warnings.  |
   |  |   | 315m 11s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.namenode.snapshot.TestSnapshot |
   |   | hadoop.tools.TestHdfsConfigFields |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   |   | hadoop.hdfs.server.namenode.TestNameNodeRetryCacheMetrics |
   |   | hadoop.hdfs.server.namenode.snapshot.TestSnapshotRename |
   |   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
   |   | hadoop.hdfs.server.namenode.snapshot.TestRenameWithSnapshots |
   |   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   |   | hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots |
   |   | hadoop.hdfs.TestReconstructStripedFileWithRandomECPolicy |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/8/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2176 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 6b2713f6389f 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 0277856738a |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/8/artifact/out/diff-compile-cc-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/8/artifact/out/diff-compile-cc-root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/8/artifact/out/diff-checkstyle-root.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/8/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/8/testReport/ |
   | Max. process+thread count | 3649 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/8/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] bshashikant commented on a change in pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#discussion_r465483195



##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
##########
@@ -244,6 +244,10 @@
       "dfs.namenode.snapshot.capture.openfiles";
   boolean DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES_DEFAULT = false;
 
+  String DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED =
+      "dfs.namenode.snapshot.trashroot.enabled";
+  boolean DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED_DEFAULT = false;
+
   String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS =

Review comment:
       u may also want to add it in hdfs-default.xml or want to maintain it as an internal config? This ls leading to a test failure TestHdfsConfigFields.testCompareConfigurationClassAgainstXml.
   
   I think its better to hide it and define the config in SnapshotManager itself similar to "dfs.namenode.snapshot.deletion.ordered"




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-669913635


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 21s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  1s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 21s |  Maven dependency ordering for branch  |
   | -1 :x: |  mvninstall  |  28m 26s |  root in trunk failed.  |
   | -1 :x: |  compile  |  13m 23s |  root in trunk failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  compile  |  10m 47s |  root in trunk failed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09.  |
   | -0 :warning: |  checkstyle  |   2m 29s |  The patch fails to run checkstyle in root  |
   | +1 :green_heart: |  mvnsite  |   3m 37s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 23s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   3m 23s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +0 :ok: |  spotbugs  |   2m 30s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   7m 36s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 52s |  the patch passed  |
   | -1 :x: |  compile  |  13m 19s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  cc  |  13m 19s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  javac  |  13m 19s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | +1 :green_heart: |  compile  |  17m 21s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | -1 :x: |  cc  |  17m 21s |  root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09 with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09 generated 129 new + 33 unchanged - 0 fixed = 162 total (was 33)  |
   | -1 :x: |  javac  |  17m 21s |  root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09 with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09 generated 192 new + 1756 unchanged - 0 fixed = 1948 total (was 1756)  |
   | -0 :warning: |  checkstyle  |   2m 40s |  The patch fails to run checkstyle in root  |
   | +1 :green_heart: |  mvnsite  |   4m  0s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m  3s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 52s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   4m  3s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | -1 :x: |  findbugs  |   2m 51s |  hadoop-hdfs-project/hadoop-hdfs-client generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 32s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 18s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  |  97m  9s |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  9s |  The patch does not generate ASF License warnings.  |
   |  |   | 274m 37s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-hdfs-project/hadoop-hdfs-client |
   |  |  Load of known null value in org.apache.hadoop.hdfs.DistributedFileSystem.getTrashRoot(Path)  At DistributedFileSystem.java:in org.apache.hadoop.hdfs.DistributedFileSystem.getTrashRoot(Path)  At DistributedFileSystem.java:[line 3292] |
   | Failed junit tests | hadoop.hdfs.TestGetFileChecksum |
   |   | hadoop.hdfs.server.namenode.ha.TestHAAppend |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   |   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   |   | hadoop.tools.TestHdfsConfigFields |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2176 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 58fae0238d0e 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / c7e71a6c0be |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | mvninstall | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/branch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/branch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/branch-compile-root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/buildtool-branch-checkstyle-root.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/diff-compile-cc-root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/diff-compile-javac-root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/buildtool-patch-checkstyle-root.txt |
   | findbugs | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/new-findbugs-hadoop-hdfs-project_hadoop-hdfs-client.html |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/testReport/ |
   | Max. process+thread count | 4450 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/9/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] smengcl commented on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-672032459


   Reran the test. Much less unrelated flaky test failures now.
   
   I'm merging the closing this PR in a minute.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-671694801


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  34m  9s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  0s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  48m 25s |  trunk passed  |
   | +1 :green_heart: |  compile  |  25m 45s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  21m 33s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   3m 41s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   4m 22s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  24m 54s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 28s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   3m 33s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   2m 36s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   8m  4s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 49s |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 10s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | -1 :x: |  cc  |  20m 10s |  root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 27 new + 135 unchanged - 27 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  20m 10s |  the patch passed  |
   | +1 :green_heart: |  compile  |  17m 42s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | -1 :x: |  cc  |  17m 42s |  root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 generated 25 new + 137 unchanged - 25 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  17m 42s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   3m 35s |  root: The patch generated 1 new + 826 unchanged - 0 fixed = 827 total (was 826)  |
   | +1 :green_heart: |  mvnsite  |   3m 47s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  16m  9s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 26s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   3m 36s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   8m 24s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 21s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  9s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  | 121m 17s |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 57s |  The patch does not generate ASF License warnings.  |
   |  |   | 387m 35s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   |   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
   |   | hadoop.hdfs.server.namenode.TestNameNodeRetryCacheMetrics |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   |   | hadoop.hdfs.TestSafeModeWithStripedFileWithRandomECPolicy |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/14/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2176 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 5425af46f42b 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 5e0f8797790 |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/14/artifact/out/diff-compile-cc-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/14/artifact/out/diff-compile-cc-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/14/artifact/out/diff-checkstyle-root.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/14/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/14/testReport/ |
   | Max. process+thread count | 2650 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/14/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] smengcl commented on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-692330196


   > I'm late to this. But we should verify to make sure httpfs also behaves the sames. Otherwise it'll break Hue.
   
   hey @jojochuang, thanks for checking in!
   
   I have written a simple test for verifying HttpFS (over WebHDFS). HttpFS works as expected. But I haven't figured out how to add a config to the HDFS cluster that `BaseTestHttpFSWith` launches.
   Currently the config change is a bit hacky. Ergo, I haven't submitted a patch for new jira to add this test. I would file a new jira to add it once this little problem is solved.
   
   [HDFS-15492.TestHttpFS.001.patch.txt](https://github.com/apache/hadoop/files/5221189/HDFS-15492.TestHttpFS.001.patch.txt)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] smengcl edited a comment on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
smengcl edited a comment on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-692330196


   > I'm late to this. But we should verify to make sure httpfs also behaves the sames. Otherwise it'll break Hue.
   
   hey @jojochuang, thanks for checking in!
   
   I have written a simple test for verifying HttpFS (over WebHDFS). HttpFS works as expected. But I haven't figured out how to add a config to the HDFS cluster that `BaseTestHttpFSWith` launches. Currently the config change for this test is a bit hacky. Ergo, I haven't submitted a patch for new jira to add this test. I would file a new jira to add it once this little problem is solved.
   
   [HDFS-15492.TestHttpFS.001.patch.txt](https://github.com/apache/hadoop/files/5221189/HDFS-15492.TestHttpFS.001.patch.txt)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] bshashikant commented on a change in pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#discussion_r466525092



##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
##########
@@ -516,6 +516,11 @@
   public static final int
       DFS_NAMENODE_SNAPSHOT_SKIPLIST_MAX_SKIP_LEVELS_DEFAULT = 0;
 
+  public static final String DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED =
+      "dfs.namenode.snapshot.trashroot.enabled";
+  public static final boolean DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED_DEFAULT =

Review comment:
       i guess we need to define these configs in SnapshotManager if we intend not add it in hdfs-default.xml(which i would prefer). It leads to test failure here "hadoop.tools.TestHdfsConfigFields"




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-670804682


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  0s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  26m 42s |  trunk passed  |
   | +1 :green_heart: |  compile  |  19m 26s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  16m 51s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  checkstyle  |   3m  7s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   4m  5s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 42s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 53s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   4m  0s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +0 :ok: |  spotbugs  |   2m 35s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   7m 50s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 45s |  the patch passed  |
   | +1 :green_heart: |  compile  |  18m 45s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | -1 :x: |  cc  |  18m 45s |  root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 11 new + 151 unchanged - 11 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  18m 45s |  the patch passed  |
   | +1 :green_heart: |  compile  |  16m 46s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | -1 :x: |  cc  |  16m 46s |  root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09 with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09 generated 25 new + 137 unchanged - 25 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  16m 46s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 59s |  root: The patch generated 1 new + 826 unchanged - 0 fixed = 827 total (was 826)  |
   | +1 :green_heart: |  mvnsite  |   4m  6s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m  1s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 53s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   4m  0s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  findbugs  |   8m 22s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 23s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 20s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  | 114m 52s |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  7s |  The patch does not generate ASF License warnings.  |
   |  |   | 309m 49s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.TestDFSStorageStateRecovery |
   |   | hadoop.hdfs.server.namenode.TestFSEditLogLoader |
   |   | hadoop.hdfs.server.namenode.ha.TestRetryCacheWithHA |
   |   | hadoop.hdfs.TestReconstructStripedFile |
   |   | hadoop.hdfs.TestDecommissionWithStripedBackoffMonitor |
   |   | hadoop.hdfs.server.namenode.TestQuotaByStorageType |
   |   | hadoop.hdfs.server.namenode.ha.TestStandbyInProgressTail |
   |   | hadoop.hdfs.server.namenode.TestFSDirectory |
   |   | hadoop.hdfs.server.namenode.TestFsck |
   |   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
   |   | hadoop.hdfs.server.namenode.ha.TestBootstrapAliasmap |
   |   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   |   | hadoop.hdfs.server.datanode.TestDataNodeErasureCodingMetrics |
   |   | hadoop.hdfs.server.datanode.TestBPOfferService |
   |   | hadoop.hdfs.server.namenode.TestNamenodeRetryCache |
   |   | hadoop.hdfs.TestErasureCodeBenchmarkThroughput |
   |   | hadoop.hdfs.server.namenode.TestBlockPlacementPolicyRackFaultTolerant |
   |   | hadoop.hdfs.server.namenode.ha.TestHASafeMode |
   |   | hadoop.hdfs.server.namenode.ha.TestPipelinesFailover |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/13/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2176 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 1720ab99aba2 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 975b6024dd9 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/13/artifact/out/diff-compile-cc-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/13/artifact/out/diff-compile-cc-root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/13/artifact/out/diff-checkstyle-root.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/13/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/13/testReport/ |
   | Max. process+thread count | 4146 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/13/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] smengcl commented on a change in pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#discussion_r467189530



##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
##########
@@ -2144,4 +2146,293 @@ public void testECCloseCommittedBlock() throws Exception {
       LambdaTestUtils.intercept(IOException.class, "", () -> str.close());
     }
   }
+
+  @Test
+  public void testGetTrashRoot() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+
+      Path trBeforeAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trBeforeAllowSnapshotStr = trBeforeAllowSnapshot.toUri().getPath();
+      // The trash root should be in user home directory
+      String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+      assertTrue(trBeforeAllowSnapshotStr.startsWith(homeDirStr));
+
+      dfs.allowSnapshot(testDir);
+
+      Path trAfterAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trAfterAllowSnapshotStr = trAfterAllowSnapshot.toUri().getPath();
+      // The trash root should now be in the snapshot root
+      String testDirStr = testDir.toUri().getPath();
+      assertTrue(trAfterAllowSnapshotStr.startsWith(testDirStr));
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private boolean isPathInUserHome(String pathStr, DistributedFileSystem dfs) {
+    String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+    return pathStr.startsWith(homeDirStr);
+  }
+
+  @Test
+  public void testGetTrashRoots() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+      // Create user trash
+      Path currUserHome = dfs.getHomeDirectory();
+      Path currUserTrash = new Path(currUserHome, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(currUserTrash);
+      // Create trash inside test directory
+      Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      Path testDirTrashCurrUser = new Path(testDirTrash,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+      dfs.mkdirs(testDirTrashCurrUser);
+
+      Collection<FileStatus> trashRoots = dfs.getTrashRoots(false);
+      // getTrashRoots should only return 1 empty user trash in the home dir now
+      assertEquals(1, trashRoots.size());
+      FileStatus firstFileStatus = trashRoots.iterator().next();
+      String pathStr = firstFileStatus.getPath().toUri().getPath();
+      assertTrue(isPathInUserHome(pathStr, dfs));
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRoots, trashRootsAllUsers);
+
+      dfs.allowSnapshot(testDir);
+
+      Collection<FileStatus> trashRootsAfter = dfs.getTrashRoots(false);
+      // getTrashRoots should return 1 more trash root inside snapshottable dir
+      assertEquals(trashRoots.size() + 1, trashRootsAfter.size());
+      boolean foundUserHomeTrash = false;
+      boolean foundSnapDirUserTrash = false;
+      String testDirStr = testDir.toUri().getPath();
+      for (FileStatus fileStatus : trashRootsAfter) {
+        String currPathStr = fileStatus.getPath().toUri().getPath();
+        if (isPathInUserHome(currPathStr, dfs)) {
+          foundUserHomeTrash = true;
+        } else if (currPathStr.startsWith(testDirStr)) {
+          foundSnapDirUserTrash = true;
+        }
+      }
+      assertTrue(foundUserHomeTrash);
+      assertTrue(foundSnapDirUserTrash);
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAfterAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter, trashRootsAfterAllUsers);
+
+      // Create trash root for user0
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user0");
+      String user0HomeStr = DFSUtilClient.getHomeDirectory(conf, ugi);
+      Path user0Trash = new Path(user0HomeStr, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(user0Trash);
+      // allUsers flag set to false should be unaffected
+      Collection<FileStatus> trashRootsAfter2 = dfs.getTrashRoots(false);
+      assertEquals(trashRootsAfter, trashRootsAfter2);
+      // allUsers flag set to true should include new user's trash
+      trashRootsAfter2 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter.size() + 1, trashRootsAfter2.size());
+
+      // Create trash root inside the snapshottable directory for user0
+      Path testDirTrashUser0 = new Path(testDirTrash, ugi.getShortUserName());
+      dfs.mkdirs(testDirTrashUser0);
+      Collection<FileStatus> trashRootsAfter3 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter2.size() + 1, trashRootsAfter3.size());
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootsOnSnapshottableDirWithEZ()
+      throws IOException, NoSuchAlgorithmException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    // Set encryption zone config
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    // Create key for EZ
+    final KeyProvider provider =
+        cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey("key", options);
+    provider.flush();
+
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test2/");
+      dfs.mkdirs(testDir);
+      dfs.createEncryptionZone(testDir, "key");
+
+      // Create trash inside test directory
+      Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      Path testDirTrashCurrUser = new Path(testDirTrash,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+      dfs.mkdirs(testDirTrashCurrUser);
+
+      Collection<FileStatus> trashRoots = dfs.getTrashRoots(false);
+      assertEquals(1, trashRoots.size());
+      FileStatus firstFileStatus = trashRoots.iterator().next();
+      String pathStr = firstFileStatus.getPath().toUri().getPath();
+      String testDirStr = testDir.toUri().getPath();
+      assertTrue(pathStr.startsWith(testDirStr));
+
+      dfs.allowSnapshot(testDir);
+
+      Collection<FileStatus> trashRootsAfter = dfs.getTrashRoots(false);
+      // getTrashRoots should give the same result
+      assertEquals(trashRoots, trashRootsAfter);
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootOnSnapshottableDirInEZ()
+      throws IOException, NoSuchAlgorithmException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    // Set EZ config
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    // Create key for EZ
+    final KeyProvider provider =
+        cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey("key", options);
+    provider.flush();
+
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+
+      Path testDir = new Path("/ssgtr/test3ez/");
+      dfs.mkdirs(testDir);
+      dfs.createEncryptionZone(testDir, "key");
+      Path testSubD = new Path(testDir, "sssubdir");
+      Path file1Path = new Path(testSubD, "file1");
+      dfs.create(file1Path);
+
+      final Path trBefore = dfs.getTrashRoot(file1Path);
+      final String trBeforeStr = trBefore.toUri().getPath();
+      // The trash root should be directly under testDir
+      final Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      final String testDirTrashStr = testDirTrash.toUri().getPath();
+      assertTrue(trBeforeStr.startsWith(testDirTrashStr));
+
+      dfs.allowSnapshot(testSubD);
+      final Path trAfter = dfs.getTrashRoot(file1Path);
+      final String trAfterStr = trAfter.toUri().getPath();
+      // The trash is now located in the dir inside
+      final Path testSubDirTrash = new Path(testSubD, FileSystem.TRASH_PREFIX);
+      final String testSubDirTrashStr = testSubDirTrash.toUri().getPath();
+      assertTrue(trAfterStr.startsWith(testSubDirTrashStr));

Review comment:
       sure. done

##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
##########
@@ -2144,4 +2146,293 @@ public void testECCloseCommittedBlock() throws Exception {
       LambdaTestUtils.intercept(IOException.class, "", () -> str.close());
     }
   }
+
+  @Test
+  public void testGetTrashRoot() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+
+      Path trBeforeAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trBeforeAllowSnapshotStr = trBeforeAllowSnapshot.toUri().getPath();
+      // The trash root should be in user home directory
+      String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+      assertTrue(trBeforeAllowSnapshotStr.startsWith(homeDirStr));
+
+      dfs.allowSnapshot(testDir);
+
+      Path trAfterAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trAfterAllowSnapshotStr = trAfterAllowSnapshot.toUri().getPath();
+      // The trash root should now be in the snapshot root
+      String testDirStr = testDir.toUri().getPath();
+      assertTrue(trAfterAllowSnapshotStr.startsWith(testDirStr));
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private boolean isPathInUserHome(String pathStr, DistributedFileSystem dfs) {
+    String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+    return pathStr.startsWith(homeDirStr);
+  }
+
+  @Test
+  public void testGetTrashRoots() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+      // Create user trash
+      Path currUserHome = dfs.getHomeDirectory();
+      Path currUserTrash = new Path(currUserHome, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(currUserTrash);
+      // Create trash inside test directory
+      Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      Path testDirTrashCurrUser = new Path(testDirTrash,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+      dfs.mkdirs(testDirTrashCurrUser);
+
+      Collection<FileStatus> trashRoots = dfs.getTrashRoots(false);
+      // getTrashRoots should only return 1 empty user trash in the home dir now
+      assertEquals(1, trashRoots.size());
+      FileStatus firstFileStatus = trashRoots.iterator().next();
+      String pathStr = firstFileStatus.getPath().toUri().getPath();
+      assertTrue(isPathInUserHome(pathStr, dfs));
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRoots, trashRootsAllUsers);
+
+      dfs.allowSnapshot(testDir);
+
+      Collection<FileStatus> trashRootsAfter = dfs.getTrashRoots(false);
+      // getTrashRoots should return 1 more trash root inside snapshottable dir
+      assertEquals(trashRoots.size() + 1, trashRootsAfter.size());
+      boolean foundUserHomeTrash = false;
+      boolean foundSnapDirUserTrash = false;
+      String testDirStr = testDir.toUri().getPath();
+      for (FileStatus fileStatus : trashRootsAfter) {
+        String currPathStr = fileStatus.getPath().toUri().getPath();
+        if (isPathInUserHome(currPathStr, dfs)) {
+          foundUserHomeTrash = true;
+        } else if (currPathStr.startsWith(testDirStr)) {
+          foundSnapDirUserTrash = true;
+        }
+      }
+      assertTrue(foundUserHomeTrash);
+      assertTrue(foundSnapDirUserTrash);
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAfterAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter, trashRootsAfterAllUsers);
+
+      // Create trash root for user0
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user0");
+      String user0HomeStr = DFSUtilClient.getHomeDirectory(conf, ugi);
+      Path user0Trash = new Path(user0HomeStr, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(user0Trash);
+      // allUsers flag set to false should be unaffected
+      Collection<FileStatus> trashRootsAfter2 = dfs.getTrashRoots(false);
+      assertEquals(trashRootsAfter, trashRootsAfter2);
+      // allUsers flag set to true should include new user's trash
+      trashRootsAfter2 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter.size() + 1, trashRootsAfter2.size());
+
+      // Create trash root inside the snapshottable directory for user0
+      Path testDirTrashUser0 = new Path(testDirTrash, ugi.getShortUserName());
+      dfs.mkdirs(testDirTrashUser0);
+      Collection<FileStatus> trashRootsAfter3 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter2.size() + 1, trashRootsAfter3.size());
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootsOnSnapshottableDirWithEZ()
+      throws IOException, NoSuchAlgorithmException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    // Set encryption zone config
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    // Create key for EZ
+    final KeyProvider provider =
+        cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey("key", options);
+    provider.flush();
+
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test2/");
+      dfs.mkdirs(testDir);
+      dfs.createEncryptionZone(testDir, "key");
+
+      // Create trash inside test directory
+      Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      Path testDirTrashCurrUser = new Path(testDirTrash,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+      dfs.mkdirs(testDirTrashCurrUser);
+
+      Collection<FileStatus> trashRoots = dfs.getTrashRoots(false);
+      assertEquals(1, trashRoots.size());
+      FileStatus firstFileStatus = trashRoots.iterator().next();
+      String pathStr = firstFileStatus.getPath().toUri().getPath();
+      String testDirStr = testDir.toUri().getPath();
+      assertTrue(pathStr.startsWith(testDirStr));
+
+      dfs.allowSnapshot(testDir);
+
+      Collection<FileStatus> trashRootsAfter = dfs.getTrashRoots(false);
+      // getTrashRoots should give the same result
+      assertEquals(trashRoots, trashRootsAfter);
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootOnSnapshottableDirInEZ()
+      throws IOException, NoSuchAlgorithmException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    // Set EZ config
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    // Create key for EZ
+    final KeyProvider provider =
+        cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey("key", options);
+    provider.flush();
+
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+
+      Path testDir = new Path("/ssgtr/test3ez/");
+      dfs.mkdirs(testDir);
+      dfs.createEncryptionZone(testDir, "key");
+      Path testSubD = new Path(testDir, "sssubdir");
+      Path file1Path = new Path(testSubD, "file1");
+      dfs.create(file1Path);
+
+      final Path trBefore = dfs.getTrashRoot(file1Path);
+      final String trBeforeStr = trBefore.toUri().getPath();
+      // The trash root should be directly under testDir
+      final Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      final String testDirTrashStr = testDirTrash.toUri().getPath();
+      assertTrue(trBeforeStr.startsWith(testDirTrashStr));
+
+      dfs.allowSnapshot(testSubD);
+      final Path trAfter = dfs.getTrashRoot(file1Path);
+      final String trAfterStr = trAfter.toUri().getPath();
+      // The trash is now located in the dir inside
+      final Path testSubDirTrash = new Path(testSubD, FileSystem.TRASH_PREFIX);
+      final String testSubDirTrashStr = testSubDirTrash.toUri().getPath();
+      assertTrue(trAfterStr.startsWith(testSubDirTrashStr));
+
+      // Cleanup
+      dfs.disallowSnapshot(testSubD);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootOnEZInSnapshottableDir()
+      throws IOException, NoSuchAlgorithmException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    // Set EZ config
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    // Create key for EZ
+    final KeyProvider provider =
+        cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey("key", options);
+    provider.flush();
+
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+
+      Path testDir = new Path("/ssgtr/test3ss/");
+      dfs.mkdirs(testDir);
+      dfs.allowSnapshot(testDir);
+      Path testSubD = new Path(testDir, "ezsubdir");
+      dfs.mkdirs(testSubD);
+      Path file1Path = new Path(testSubD, "file1");
+      dfs.create(file1Path);
+
+      final Path trBefore = dfs.getTrashRoot(file1Path);
+      final String trBeforeStr = trBefore.toUri().getPath();
+      // The trash root should be directly under testDir
+      final Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      final String testDirTrashStr = testDirTrash.toUri().getPath();
+      assertTrue(trBeforeStr.startsWith(testDirTrashStr));
+
+      // Need to remove the file inside the dir to establish EZ
+      dfs.delete(file1Path, false);
+      dfs.createEncryptionZone(testSubD, "key");
+      dfs.create(file1Path);
+
+      final Path trAfter = dfs.getTrashRoot(file1Path);
+      final String trAfterStr = trAfter.toUri().getPath();
+      // The trash is now located in the dir inside
+      final Path testSubDirTrash = new Path(testSubD, FileSystem.TRASH_PREFIX);
+      final String testSubDirTrashStr = testSubDirTrash.toUri().getPath();
+      assertTrue(trAfterStr.startsWith(testSubDirTrashStr));
+

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2176: [WIP] HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-665929967






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] smengcl commented on a change in pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#discussion_r465975831



##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
##########
@@ -244,6 +244,10 @@
       "dfs.namenode.snapshot.capture.openfiles";
   boolean DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES_DEFAULT = false;
 
+  String DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED =
+      "dfs.namenode.snapshot.trashroot.enabled";
+  boolean DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED_DEFAULT = false;
+
   String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS =

Review comment:
       makes sense. will hide it from the client




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] bshashikant commented on a change in pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#discussion_r466518339



##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
##########
@@ -2144,4 +2146,293 @@ public void testECCloseCommittedBlock() throws Exception {
       LambdaTestUtils.intercept(IOException.class, "", () -> str.close());
     }
   }
+
+  @Test
+  public void testGetTrashRoot() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+
+      Path trBeforeAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trBeforeAllowSnapshotStr = trBeforeAllowSnapshot.toUri().getPath();
+      // The trash root should be in user home directory
+      String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+      assertTrue(trBeforeAllowSnapshotStr.startsWith(homeDirStr));
+
+      dfs.allowSnapshot(testDir);
+
+      Path trAfterAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trAfterAllowSnapshotStr = trAfterAllowSnapshot.toUri().getPath();
+      // The trash root should now be in the snapshot root
+      String testDirStr = testDir.toUri().getPath();
+      assertTrue(trAfterAllowSnapshotStr.startsWith(testDirStr));
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private boolean isPathInUserHome(String pathStr, DistributedFileSystem dfs) {
+    String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+    return pathStr.startsWith(homeDirStr);
+  }
+
+  @Test
+  public void testGetTrashRoots() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+      // Create user trash
+      Path currUserHome = dfs.getHomeDirectory();
+      Path currUserTrash = new Path(currUserHome, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(currUserTrash);
+      // Create trash inside test directory
+      Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      Path testDirTrashCurrUser = new Path(testDirTrash,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+      dfs.mkdirs(testDirTrashCurrUser);
+
+      Collection<FileStatus> trashRoots = dfs.getTrashRoots(false);
+      // getTrashRoots should only return 1 empty user trash in the home dir now
+      assertEquals(1, trashRoots.size());
+      FileStatus firstFileStatus = trashRoots.iterator().next();
+      String pathStr = firstFileStatus.getPath().toUri().getPath();
+      assertTrue(isPathInUserHome(pathStr, dfs));
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRoots, trashRootsAllUsers);
+
+      dfs.allowSnapshot(testDir);
+
+      Collection<FileStatus> trashRootsAfter = dfs.getTrashRoots(false);
+      // getTrashRoots should return 1 more trash root inside snapshottable dir
+      assertEquals(trashRoots.size() + 1, trashRootsAfter.size());
+      boolean foundUserHomeTrash = false;
+      boolean foundSnapDirUserTrash = false;
+      String testDirStr = testDir.toUri().getPath();
+      for (FileStatus fileStatus : trashRootsAfter) {
+        String currPathStr = fileStatus.getPath().toUri().getPath();
+        if (isPathInUserHome(currPathStr, dfs)) {
+          foundUserHomeTrash = true;
+        } else if (currPathStr.startsWith(testDirStr)) {
+          foundSnapDirUserTrash = true;
+        }
+      }
+      assertTrue(foundUserHomeTrash);
+      assertTrue(foundSnapDirUserTrash);
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAfterAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter, trashRootsAfterAllUsers);
+
+      // Create trash root for user0
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user0");
+      String user0HomeStr = DFSUtilClient.getHomeDirectory(conf, ugi);
+      Path user0Trash = new Path(user0HomeStr, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(user0Trash);
+      // allUsers flag set to false should be unaffected
+      Collection<FileStatus> trashRootsAfter2 = dfs.getTrashRoots(false);
+      assertEquals(trashRootsAfter, trashRootsAfter2);
+      // allUsers flag set to true should include new user's trash
+      trashRootsAfter2 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter.size() + 1, trashRootsAfter2.size());
+
+      // Create trash root inside the snapshottable directory for user0
+      Path testDirTrashUser0 = new Path(testDirTrash, ugi.getShortUserName());
+      dfs.mkdirs(testDirTrashUser0);
+      Collection<FileStatus> trashRootsAfter3 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter2.size() + 1, trashRootsAfter3.size());
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootsOnSnapshottableDirWithEZ()
+      throws IOException, NoSuchAlgorithmException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    // Set encryption zone config
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    // Create key for EZ
+    final KeyProvider provider =
+        cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey("key", options);
+    provider.flush();
+
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test2/");
+      dfs.mkdirs(testDir);
+      dfs.createEncryptionZone(testDir, "key");
+
+      // Create trash inside test directory
+      Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      Path testDirTrashCurrUser = new Path(testDirTrash,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+      dfs.mkdirs(testDirTrashCurrUser);
+
+      Collection<FileStatus> trashRoots = dfs.getTrashRoots(false);
+      assertEquals(1, trashRoots.size());
+      FileStatus firstFileStatus = trashRoots.iterator().next();
+      String pathStr = firstFileStatus.getPath().toUri().getPath();
+      String testDirStr = testDir.toUri().getPath();
+      assertTrue(pathStr.startsWith(testDirStr));
+
+      dfs.allowSnapshot(testDir);
+
+      Collection<FileStatus> trashRootsAfter = dfs.getTrashRoots(false);
+      // getTrashRoots should give the same result
+      assertEquals(trashRoots, trashRootsAfter);
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootOnSnapshottableDirInEZ()
+      throws IOException, NoSuchAlgorithmException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    // Set EZ config
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    // Create key for EZ
+    final KeyProvider provider =
+        cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey("key", options);
+    provider.flush();
+
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+
+      Path testDir = new Path("/ssgtr/test3ez/");
+      dfs.mkdirs(testDir);
+      dfs.createEncryptionZone(testDir, "key");
+      Path testSubD = new Path(testDir, "sssubdir");
+      Path file1Path = new Path(testSubD, "file1");
+      dfs.create(file1Path);
+
+      final Path trBefore = dfs.getTrashRoot(file1Path);
+      final String trBeforeStr = trBefore.toUri().getPath();
+      // The trash root should be directly under testDir
+      final Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      final String testDirTrashStr = testDirTrash.toUri().getPath();
+      assertTrue(trBeforeStr.startsWith(testDirTrashStr));
+
+      dfs.allowSnapshot(testSubD);
+      final Path trAfter = dfs.getTrashRoot(file1Path);
+      final String trAfterStr = trAfter.toUri().getPath();
+      // The trash is now located in the dir inside
+      final Path testSubDirTrash = new Path(testSubD, FileSystem.TRASH_PREFIX);
+      final String testSubDirTrashStr = testSubDirTrash.toUri().getPath();
+      assertTrue(trAfterStr.startsWith(testSubDirTrashStr));
+
+      // Cleanup
+      dfs.disallowSnapshot(testSubD);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootOnEZInSnapshottableDir()
+      throws IOException, NoSuchAlgorithmException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    // Set EZ config
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    // Create key for EZ
+    final KeyProvider provider =
+        cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey("key", options);
+    provider.flush();
+
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+
+      Path testDir = new Path("/ssgtr/test3ss/");
+      dfs.mkdirs(testDir);
+      dfs.allowSnapshot(testDir);
+      Path testSubD = new Path(testDir, "ezsubdir");
+      dfs.mkdirs(testSubD);
+      Path file1Path = new Path(testSubD, "file1");
+      dfs.create(file1Path);
+
+      final Path trBefore = dfs.getTrashRoot(file1Path);
+      final String trBeforeStr = trBefore.toUri().getPath();
+      // The trash root should be directly under testDir
+      final Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      final String testDirTrashStr = testDirTrash.toUri().getPath();
+      assertTrue(trBeforeStr.startsWith(testDirTrashStr));
+
+      // Need to remove the file inside the dir to establish EZ
+      dfs.delete(file1Path, false);
+      dfs.createEncryptionZone(testSubD, "key");
+      dfs.create(file1Path);
+
+      final Path trAfter = dfs.getTrashRoot(file1Path);
+      final String trAfterStr = trAfter.toUri().getPath();
+      // The trash is now located in the dir inside
+      final Path testSubDirTrash = new Path(testSubD, FileSystem.TRASH_PREFIX);
+      final String testSubDirTrashStr = testSubDirTrash.toUri().getPath();
+      assertTrue(trAfterStr.startsWith(testSubDirTrashStr));
+

Review comment:
       startsWith --> equals




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] smengcl commented on a change in pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#discussion_r466231836



##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
##########
@@ -2144,4 +2146,180 @@ public void testECCloseCommittedBlock() throws Exception {
       LambdaTestUtils.intercept(IOException.class, "", () -> str.close());
     }
   }
+
+  @Test
+  public void testGetTrashRoot() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+
+      Path trBeforeAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trBeforeAllowSnapshotStr = trBeforeAllowSnapshot.toUri().getPath();
+      // The trash root should be in user home directory
+      String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+      assertTrue(trBeforeAllowSnapshotStr.startsWith(homeDirStr));
+
+      dfs.allowSnapshot(testDir);
+
+      Path trAfterAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trAfterAllowSnapshotStr = trAfterAllowSnapshot.toUri().getPath();
+      // The trash root should now be in the snapshot root
+      String testDirStr = testDir.toUri().getPath();
+      assertTrue(trAfterAllowSnapshotStr.startsWith(testDirStr));
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private boolean isPathInUserHome(String pathStr, DistributedFileSystem dfs) {
+    String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+    return pathStr.startsWith(homeDirStr);
+  }
+
+  @Test
+  public void testGetTrashRoots() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+      // Create user trash
+      Path currUserHome = dfs.getHomeDirectory();
+      Path currUserTrash = new Path(currUserHome, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(currUserTrash);
+      // Create trash inside test directory
+      Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      Path testDirTrashCurrUser = new Path(testDirTrash,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+      dfs.mkdirs(testDirTrashCurrUser);
+
+      Collection<FileStatus> trashRoots = dfs.getTrashRoots(false);
+      // getTrashRoots should only return 1 empty user trash in the home dir now
+      assertEquals(1, trashRoots.size());
+      FileStatus firstFileStatus = trashRoots.iterator().next();
+      String pathStr = firstFileStatus.getPath().toUri().getPath();
+      assertTrue(isPathInUserHome(pathStr, dfs));
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRoots, trashRootsAllUsers);
+
+      dfs.allowSnapshot(testDir);
+
+      Collection<FileStatus> trashRootsAfter = dfs.getTrashRoots(false);
+      // getTrashRoots should return 1 more trash root inside snapshottable dir
+      assertEquals(trashRoots.size() + 1, trashRootsAfter.size());
+      boolean foundUserHomeTrash = false;
+      boolean foundSnapDirUserTrash = false;
+      String testDirStr = testDir.toUri().getPath();
+      for (FileStatus fileStatus : trashRootsAfter) {
+        String currPathStr = fileStatus.getPath().toUri().getPath();
+        if (isPathInUserHome(currPathStr, dfs)) {
+          foundUserHomeTrash = true;
+        } else if (currPathStr.startsWith(testDirStr)) {
+          foundSnapDirUserTrash = true;
+        }
+      }
+      assertTrue(foundUserHomeTrash);
+      assertTrue(foundSnapDirUserTrash);
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAfterAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter, trashRootsAfterAllUsers);
+
+      // Create trash root for user0
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user0");
+      String user0HomeStr = DFSUtilClient.getHomeDirectory(conf, ugi);
+      Path user0Trash = new Path(user0HomeStr, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(user0Trash);
+      // allUsers flag set to false should be unaffected
+      Collection<FileStatus> trashRootsAfter2 = dfs.getTrashRoots(false);
+      assertEquals(trashRootsAfter, trashRootsAfter2);
+      // allUsers flag set to true should include new user's trash
+      trashRootsAfter2 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter.size() + 1, trashRootsAfter2.size());
+
+      // Create trash root inside the snapshottable directory for user0
+      Path testDirTrashUser0 = new Path(testDirTrash, ugi.getShortUserName());
+      dfs.mkdirs(testDirTrashUser0);
+      Collection<FileStatus> trashRootsAfter3 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter2.size() + 1, trashRootsAfter3.size());
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootsOnSnapshottableDirWithEncryptionZone()

Review comment:
       With a288110174c55b2c601b79b2beca9a27b2f34102, when a path given to getTrashRoot() is both inside an EZ and in a snapshottable dir, it should now choose the inner most trash.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-668563963


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 14s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  0s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  19m 15s |  trunk passed  |
   | +1 :green_heart: |  compile  |  21m 24s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  18m 12s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  checkstyle  |   2m 56s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   4m  9s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 48s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 35s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   3m 57s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +0 :ok: |  spotbugs  |   3m 23s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   8m 17s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m  1s |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 25s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | -1 :x: |  cc  |  20m 25s |  root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 30 new + 132 unchanged - 30 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  20m 25s |  the patch passed  |
   | +1 :green_heart: |  compile  |  18m 32s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | -1 :x: |  cc  |  18m 32s |  root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09 with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09 generated 36 new + 126 unchanged - 36 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  18m 32s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   3m  4s |  root: The patch generated 3 new + 826 unchanged - 0 fixed = 829 total (was 826)  |
   | +1 :green_heart: |  mvnsite  |   4m 13s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 41s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 37s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   3m 54s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  findbugs  |   9m  5s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 37s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 18s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  | 150m 21s |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  6s |  The patch does not generate ASF License warnings.  |
   |  |   | 343m 53s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.tools.TestHdfsConfigFields |
   |   | hadoop.hdfs.TestStripedFileAppend |
   |   | hadoop.hdfs.TestDecommissionWithStriped |
   |   | hadoop.hdfs.TestFileChecksum |
   |   | hadoop.hdfs.TestAclsEndToEnd |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   |   | hadoop.hdfs.TestMaintenanceState |
   |   | hadoop.hdfs.server.namenode.TestNameNodeRetryCacheMetrics |
   |   | hadoop.hdfs.TestLeaseRecovery2 |
   |   | hadoop.hdfs.TestDFSStripedInputStreamWithRandomECPolicy |
   |   | hadoop.hdfs.TestDFSClientFailover |
   |   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   |   | hadoop.hdfs.TestReservedRawPaths |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2176 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 1e62ed8ab7a6 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / ab2b3df2de1 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/6/artifact/out/diff-compile-cc-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/6/artifact/out/diff-compile-cc-root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/6/artifact/out/diff-checkstyle-root.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/6/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/6/testReport/ |
   | Max. process+thread count | 3801 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/6/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
jojochuang commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-682989066


   I'm late to this. But we should verify to make sure httpfs also behaves the sames. Otherwise it'll break Hue.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-669936881


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  26m  0s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  0s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 21s |  Maven dependency ordering for branch  |
   | -1 :x: |  mvninstall  |  25m  8s |  root in trunk failed.  |
   | -1 :x: |  compile  |  12m 40s |  root in trunk failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  compile  |  10m 40s |  root in trunk failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   2m 31s |  The patch fails to run checkstyle in root  |
   | +1 :green_heart: |  mvnsite  |   3m 32s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 42s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 17s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   3m 22s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   2m 38s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   7m 44s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 46s |  the patch passed  |
   | -1 :x: |  compile  |  12m 46s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  cc  |  12m 46s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  javac  |  12m 46s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  compile  |  11m 48s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -1 :x: |  cc  |  11m 48s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -1 :x: |  javac  |  11m 48s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   2m 40s |  The patch fails to run checkstyle in root  |
   | +1 :green_heart: |  mvnsite  |   3m 27s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 32s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 11s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   3m 11s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | -1 :x: |  findbugs  |   2m 31s |  hadoop-hdfs-project/hadoop-hdfs-client generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 40s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  6s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  |  99m  3s |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 47s |  The patch does not generate ASF License warnings.  |
   |  |   | 287m 15s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-hdfs-project/hadoop-hdfs-client |
   |  |  Load of known null value in org.apache.hadoop.hdfs.DistributedFileSystem.getTrashRoot(Path)  At DistributedFileSystem.java:in org.apache.hadoop.hdfs.DistributedFileSystem.getTrashRoot(Path)  At DistributedFileSystem.java:[line 3292] |
   | Failed junit tests | hadoop.hdfs.server.namenode.TestNameNodeRetryCacheMetrics |
   |   | hadoop.hdfs.server.namenode.TestAddOverReplicatedStripedBlocks |
   |   | hadoop.tools.TestHdfsConfigFields |
   |   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   |   | hadoop.hdfs.server.namenode.TestDecommissioningStatus |
   |   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
   |   | hadoop.hdfs.TestRollingUpgrade |
   |   | hadoop.hdfs.server.datanode.TestBlockScanner |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2176 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 8f8155b72828 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / c7e71a6c0be |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | mvninstall | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/branch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/branch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/branch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/buildtool-branch-checkstyle-root.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/buildtool-patch-checkstyle-root.txt |
   | findbugs | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/new-findbugs-hadoop-hdfs-project_hadoop-hdfs-client.html |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/testReport/ |
   | Max. process+thread count | 3891 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/10/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] smengcl commented on a change in pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#discussion_r467222131



##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
##########
@@ -516,6 +516,11 @@
   public static final int
       DFS_NAMENODE_SNAPSHOT_SKIPLIST_MAX_SKIP_LEVELS_DEFAULT = 0;
 
+  public static final String DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED =
+      "dfs.namenode.snapshot.trashroot.enabled";
+  public static final boolean DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED_DEFAULT =

Review comment:
       Got it. I will put it in `FSNameSystem` as private config then as it is used there, similar to HDFS-15481 [did](https://github.com/apache/hadoop/blob/e072d33327b8f5d38b74a15e279d492ad379a47c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java#L88-L96).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] smengcl edited a comment on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
smengcl edited a comment on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-672032459


   Reran the test. Much less unrelated flaky test failures now.
   
   I'm merging this PR in a minute.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2176: [WIP] HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-666940884


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  29m  0s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | -1 :x: |  test4tests  |   0m  0s |  The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch.  |
   ||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  20m 13s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |   0m 53s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  checkstyle  |   0m 29s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 58s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 17s |  branch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 40s |  hadoop-hdfs-client in trunk failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +0 :ok: |  spotbugs  |   2m 22s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   2m 20s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 47s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 50s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javac  |   0m 50s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 43s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  javac  |   0m 43s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 19s |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 46s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 15s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 34s |  hadoop-hdfs-client in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  findbugs  |   2m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 59s |  hadoop-hdfs-client in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 29s |  The patch does not generate ASF License warnings.  |
   |  |   |  97m 42s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2176 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux e3692f3aa35a 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 05b3337a460 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | javadoc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/2/artifact/out/branch-javadoc-hadoop-hdfs-project_hadoop-hdfs-client-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | javadoc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/2/artifact/out/patch-javadoc-hadoop-hdfs-project_hadoop-hdfs-client-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/2/testReport/ |
   | Max. process+thread count | 413 (vs. ulimit of 5500) |
   | modules | C: hadoop-hdfs-project/hadoop-hdfs-client U: hadoop-hdfs-project/hadoop-hdfs-client |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/2/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#issuecomment-668518250


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  0s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 53s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  20m 18s |  trunk passed  |
   | +1 :green_heart: |  compile  |  22m 24s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  18m 47s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  checkstyle  |   2m 53s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   4m 25s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 42s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 43s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   3m 52s |  trunk passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +0 :ok: |  spotbugs  |   3m 28s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   8m 27s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 56s |  the patch passed  |
   | +1 :green_heart: |  compile  |  22m 17s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | -1 :x: |  cc  |  22m 17s |  root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 38 new + 124 unchanged - 38 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  22m 17s |  the patch passed  |
   | +1 :green_heart: |  compile  |  18m 27s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | -1 :x: |  cc  |  18m 27s |  root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09 with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09 generated 23 new + 139 unchanged - 23 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  18m 27s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 55s |  root: The patch generated 3 new + 826 unchanged - 0 fixed = 829 total (was 826)  |
   | +1 :green_heart: |  mvnsite  |   3m 56s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m  5s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 39s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   3m 49s |  the patch passed with JDK Private Build-1.8.0_252-8u252-b09-1~18.04-b09  |
   | +1 :green_heart: |  findbugs  |   8m 23s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 21s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 21s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  |  96m  9s |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  7s |  The patch does not generate ASF License warnings.  |
   |  |   | 292m 57s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.namenode.TestAddStripedBlockInFBR |
   |   | hadoop.hdfs.server.namenode.TestNameNodeRetryCacheMetrics |
   |   | hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks |
   |   | hadoop.tools.TestHdfsConfigFields |
   |   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2176 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 40a24d11891b 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / ab2b3df2de1 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/5/artifact/out/diff-compile-cc-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/5/artifact/out/diff-compile-cc-root-jdkPrivateBuild-1.8.0_252-8u252-b09-1~18.04-b09.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/5/artifact/out/diff-checkstyle-root.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/5/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/5/testReport/ |
   | Max. process+thread count | 4814 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2176/5/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] bshashikant commented on a change in pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#discussion_r465605682



##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
##########
@@ -2144,4 +2146,180 @@ public void testECCloseCommittedBlock() throws Exception {
       LambdaTestUtils.intercept(IOException.class, "", () -> str.close());
     }
   }
+
+  @Test
+  public void testGetTrashRoot() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+
+      Path trBeforeAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trBeforeAllowSnapshotStr = trBeforeAllowSnapshot.toUri().getPath();
+      // The trash root should be in user home directory
+      String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+      assertTrue(trBeforeAllowSnapshotStr.startsWith(homeDirStr));
+
+      dfs.allowSnapshot(testDir);
+
+      Path trAfterAllowSnapshot = dfs.getTrashRoot(file0path);
+      String trAfterAllowSnapshotStr = trAfterAllowSnapshot.toUri().getPath();
+      // The trash root should now be in the snapshot root
+      String testDirStr = testDir.toUri().getPath();
+      assertTrue(trAfterAllowSnapshotStr.startsWith(testDirStr));
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private boolean isPathInUserHome(String pathStr, DistributedFileSystem dfs) {
+    String homeDirStr = dfs.getHomeDirectory().toUri().getPath();
+    return pathStr.startsWith(homeDirStr);
+  }
+
+  @Test
+  public void testGetTrashRoots() throws IOException {
+    Configuration conf = getTestConfiguration();
+    conf.setBoolean(DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED, true);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      Path testDir = new Path("/ssgtr/test1/");
+      Path file0path = new Path(testDir, "file-0");
+      dfs.create(file0path);
+      // Create user trash
+      Path currUserHome = dfs.getHomeDirectory();
+      Path currUserTrash = new Path(currUserHome, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(currUserTrash);
+      // Create trash inside test directory
+      Path testDirTrash = new Path(testDir, FileSystem.TRASH_PREFIX);
+      Path testDirTrashCurrUser = new Path(testDirTrash,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+      dfs.mkdirs(testDirTrashCurrUser);
+
+      Collection<FileStatus> trashRoots = dfs.getTrashRoots(false);
+      // getTrashRoots should only return 1 empty user trash in the home dir now
+      assertEquals(1, trashRoots.size());
+      FileStatus firstFileStatus = trashRoots.iterator().next();
+      String pathStr = firstFileStatus.getPath().toUri().getPath();
+      assertTrue(isPathInUserHome(pathStr, dfs));
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRoots, trashRootsAllUsers);
+
+      dfs.allowSnapshot(testDir);
+
+      Collection<FileStatus> trashRootsAfter = dfs.getTrashRoots(false);
+      // getTrashRoots should return 1 more trash root inside snapshottable dir
+      assertEquals(trashRoots.size() + 1, trashRootsAfter.size());
+      boolean foundUserHomeTrash = false;
+      boolean foundSnapDirUserTrash = false;
+      String testDirStr = testDir.toUri().getPath();
+      for (FileStatus fileStatus : trashRootsAfter) {
+        String currPathStr = fileStatus.getPath().toUri().getPath();
+        if (isPathInUserHome(currPathStr, dfs)) {
+          foundUserHomeTrash = true;
+        } else if (currPathStr.startsWith(testDirStr)) {
+          foundSnapDirUserTrash = true;
+        }
+      }
+      assertTrue(foundUserHomeTrash);
+      assertTrue(foundSnapDirUserTrash);
+      // allUsers should not make a difference for now because we have one user
+      Collection<FileStatus> trashRootsAfterAllUsers = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter, trashRootsAfterAllUsers);
+
+      // Create trash root for user0
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user0");
+      String user0HomeStr = DFSUtilClient.getHomeDirectory(conf, ugi);
+      Path user0Trash = new Path(user0HomeStr, FileSystem.TRASH_PREFIX);
+      dfs.mkdirs(user0Trash);
+      // allUsers flag set to false should be unaffected
+      Collection<FileStatus> trashRootsAfter2 = dfs.getTrashRoots(false);
+      assertEquals(trashRootsAfter, trashRootsAfter2);
+      // allUsers flag set to true should include new user's trash
+      trashRootsAfter2 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter.size() + 1, trashRootsAfter2.size());
+
+      // Create trash root inside the snapshottable directory for user0
+      Path testDirTrashUser0 = new Path(testDirTrash, ugi.getShortUserName());
+      dfs.mkdirs(testDirTrashUser0);
+      Collection<FileStatus> trashRootsAfter3 = dfs.getTrashRoots(true);
+      assertEquals(trashRootsAfter2.size() + 1, trashRootsAfter3.size());
+
+      // Cleanup
+      dfs.disallowSnapshot(testDir);
+      dfs.delete(testDir, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testGetTrashRootsOnSnapshottableDirWithEncryptionZone()

Review comment:
       Let's say we have a structure like /dir1/dir2 where dir1 is ez enabled  and dir2 is made snapshottable. In such cases, anything deleted under dir2 will be under trash location under dir2 while everything which is deleted within dir1 but not dir2, will exist in trash under dir1. Will it lead to any issues??




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] bshashikant commented on a change in pull request #2176: HDFS-15492. Make trash root inside each snapshottable directory

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2176:
URL: https://github.com/apache/hadoop/pull/2176#discussion_r466152617



##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
##########
@@ -244,6 +244,10 @@
       "dfs.namenode.snapshot.capture.openfiles";
   boolean DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES_DEFAULT = false;
 
+  String DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED =
+      "dfs.namenode.snapshot.trashroot.enabled";
+  boolean DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED_DEFAULT = false;
+
   String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS =

Review comment:
       If we hide it, i would prefer to not add it in hdfs-default.xml as well. We can make this an internal config which is required only if u want to restrict renames out of snapshottable root which can be turned on only if a requirement like this arises in certain use cases.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org