فهرست منبع

HDFS-15817. Rename snapshots while marking them deleted. (#2677)

bshashikant 4 سال پیش
والد
کامیت
2df2dfb9ed

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java

@@ -61,6 +61,10 @@ public class Snapshot implements Comparable<byte[]> {
     return new SimpleDateFormat(DEFAULT_SNAPSHOT_NAME_PATTERN).format(new Date());
   }
 
+  public static String generateDeletedSnapshotName(Snapshot s) {
+    return getSnapshotName(s) + "#" + s.getId();
+  }
+
   public static String getSnapshotPath(String snapshottableDir,
       String snapshotRelativePath) {
     final StringBuilder b = new StringBuilder(snapshottableDir);

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java

@@ -66,6 +66,7 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.metrics2.util.MBeans;
 
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -533,6 +534,8 @@ public class SnapshotManager implements SnapshotStatsMXBean {
             INodesInPath.append(iip, snapshot.getRoot(),
                 DFSUtil.string2Bytes(snapshotName)), xattrs,
             EnumSet.of(XAttrSetFlag.CREATE, XAttrSetFlag.REPLACE));
+        renameSnapshot(iip, srcRoot.getFullPathName(), snapshotName,
+            Snapshot.generateDeletedSnapshotName(snapshot), Time.now());
         return;
       }
 

+ 30 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOrderedSnapshotDeletion.java

@@ -92,18 +92,25 @@ public class TestOrderedSnapshotDeletion {
     assertXAttrSet("s2", hdfs, null);
     hdfs.deleteSnapshot(snapshottableDir, "s0");
     assertXAttrSet("s2", hdfs, null);
-    hdfs.deleteSnapshot(snapshottableDir, "s1");
-    hdfs.deleteSnapshot(snapshottableDir, "s2");
+    hdfs.deleteSnapshot(snapshottableDir,
+        getDeletedSnapshotName(hdfs, snapshottableDir, "s1"));
+    hdfs.deleteSnapshot(snapshottableDir,
+        getDeletedSnapshotName(hdfs, snapshottableDir, "s2"));
   }
 
-  static void assertMarkedAsDeleted(Path snapshotRoot, MiniDFSCluster cluster)
-      throws IOException {
+  static void assertMarkedAsDeleted(Path snapshotRoot, Path snapshottableDir,
+      MiniDFSCluster cluster) throws IOException {
+    final String snapName =
+        getDeletedSnapshotName(cluster.getFileSystem(), snapshottableDir,
+            snapshotRoot.getName());
+    final Path snapPathNew =
+        SnapshotTestHelper.getSnapshotRoot(snapshottableDir, snapName);
     // Check if the path exists
-    Assert.assertNotNull(cluster.getFileSystem().getFileStatus(snapshotRoot));
+    Assert.assertNotNull(cluster.getFileSystem().getFileStatus(snapPathNew));
 
     // Check xAttr for snapshotRoot
     final INode inode = cluster.getNamesystem().getFSDirectory()
-        .getINode(snapshotRoot.toString());
+        .getINode(snapPathNew.toString());
     final XAttrFeature f = inode.getXAttrFeature();
     final XAttr xAttr = f.getXAttr(XATTR_SNAPSHOT_DELETED);
     Assert.assertNotNull(xAttr);
@@ -114,7 +121,7 @@ public class TestOrderedSnapshotDeletion {
 
     // Check inode
     Assert.assertTrue(inode instanceof Snapshot.Root);
-    Assert.assertTrue(((Snapshot.Root)inode).isMarkedAsDeleted());
+    Assert.assertTrue(((Snapshot.Root) inode).isMarkedAsDeleted());
   }
 
   static void assertNotMarkedAsDeleted(Path snapshotRoot,
@@ -139,12 +146,16 @@ public class TestOrderedSnapshotDeletion {
   void assertXAttrSet(String snapshot,
                       DistributedFileSystem hdfs, XAttr newXattr)
       throws IOException {
-    hdfs.deleteSnapshot(snapshottableDir, snapshot);
+    String snapName = getDeletedSnapshotName(hdfs, snapshottableDir, snapshot);
+    hdfs.deleteSnapshot(snapshottableDir, snapName);
     // Check xAttr for parent directory
-    Path snapshotRoot = SnapshotTestHelper.getSnapshotRoot(snapshottableDir,
-        snapshot);
-    assertMarkedAsDeleted(snapshotRoot, cluster);
-
+    Path snapshotRoot =
+        SnapshotTestHelper.getSnapshotRoot(snapshottableDir, snapshot);
+    assertMarkedAsDeleted(snapshotRoot, snapshottableDir, cluster);
+    // Check xAttr for parent directory
+    snapName = getDeletedSnapshotName(hdfs, snapshottableDir, snapshot);
+    snapshotRoot =
+        SnapshotTestHelper.getSnapshotRoot(snapshottableDir, snapName);
     // Make sure its not user visible
     if (cluster.getNameNode().getConf().getBoolean(DFSConfigKeys.
             DFS_NAMENODE_XATTRS_ENABLED_KEY,
@@ -261,4 +272,11 @@ public class TestOrderedSnapshotDeletion {
     hdfs.createSnapshot(snapshottableDir, "s1");
     assertXAttrSet("s1", hdfs, newXAttr);
   }
+
+  public static String getDeletedSnapshotName(DistributedFileSystem hdfs,
+      Path snapshottableDir, String snapshot) throws IOException {
+    return Arrays.stream(hdfs.getSnapshotListing(snapshottableDir))
+        .filter(p -> p.getFullPath().getName().startsWith(snapshot)).findFirst()
+        .get().getFullPath().getName();
+  }
 }

+ 17 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOrderedSnapshotDeletionGc.java

@@ -50,6 +50,7 @@ import static org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.DF
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion.assertMarkedAsDeleted;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion.assertNotMarkedAsDeleted;
+import static org.apache.hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion.getDeletedSnapshotName;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -112,23 +113,32 @@ public class TestOrderedSnapshotDeletionGc {
     hdfs.deleteSnapshot(snapshottableDir, "s2");
     assertNotMarkedAsDeleted(s0path, cluster);
     assertNotMarkedAsDeleted(s1path, cluster);
-    assertMarkedAsDeleted(s2path, cluster);
+    assertMarkedAsDeleted(s2path, snapshottableDir, cluster);
+    final Path s2pathNew = new Path(s2path.getParent(),
+        getDeletedSnapshotName(hdfs, snapshottableDir, s2path.getName()));
+    Assert.assertFalse(exist(s2path, hdfs));
+    Assert.assertTrue(exist(s2pathNew, hdfs));
+    Assert.assertFalse(s2path.equals(s2pathNew));
 
     hdfs.deleteSnapshot(snapshottableDir, "s1");
     assertNotMarkedAsDeleted(s0path, cluster);
-    assertMarkedAsDeleted(s1path, cluster);
-    assertMarkedAsDeleted(s2path, cluster);
-
+    assertMarkedAsDeleted(s1path, snapshottableDir, cluster);
+    assertMarkedAsDeleted(s2path, snapshottableDir, cluster);
+    final Path s1pathNew = new Path(s1path.getParent(),
+        getDeletedSnapshotName(hdfs, snapshottableDir, s1path.getName()));
+    Assert.assertFalse(exist(s1path, hdfs));
+    Assert.assertTrue(exist(s1pathNew, hdfs));
+    Assert.assertFalse(s1path.equals(s1pathNew));
     // should not be gc'ed
     Thread.sleep(10*GC_PERIOD);
     assertNotMarkedAsDeleted(s0path, cluster);
-    assertMarkedAsDeleted(s1path, cluster);
-    assertMarkedAsDeleted(s2path, cluster);
+    assertMarkedAsDeleted(s1path, snapshottableDir, cluster);
+    assertMarkedAsDeleted(s2path, snapshottableDir, cluster);
 
     hdfs.deleteSnapshot(snapshottableDir, "s0");
     Assert.assertFalse(exist(s0path, hdfs));
 
-    waitForGc(Arrays.asList(s1path, s2path), hdfs);
+    waitForGc(Arrays.asList(s1pathNew, s2pathNew), hdfs);
     // total no of edit log records created for delete snapshot will be equal
     // to sum of no of user deleted snapshots and no of snapshots gc'ed with
     // snapshotDeletion gc thread