Przeglądaj źródła

HDFS-4648. For snapshot deletion, when merging the diff from to-delete snapshot to the prior snapshot, make sure files/directories created after the prior snapshot get deleted. Contributed by Jing Zhao

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1462865 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 lat temu
rodzic
commit
9602869d19

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt

@@ -216,3 +216,7 @@ Branch-2802 Snapshot (Unreleased)
 
   HDFS-4636. Update quota usage when deleting files/dirs that were created
   after taking the latest snapshot. (Jing Zhao via szetszwo)
+
+  HDFS-4648. For snapshot deletion, when merging the diff from to-delete
+  snapshot to the prior snapshot, make sure files/directories created after
+  the prior snapshot get deleted. (Jing Zhao via szetszwo)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -622,7 +622,7 @@ public class INodeDirectory extends INodeWithAdditionalFields {
     // the diff list, the snapshot to be deleted has been combined or renamed
     // to its latest previous snapshot. (besides, we also need to consider nodes
     // created after prior but before snapshot. this will be done in 
-    // INodeDirectoryWithSnapshot#cleanSubtree
+    // INodeDirectoryWithSnapshot#cleanSubtree)
     Snapshot s = snapshot != null && prior != null ? prior : snapshot;
     for (INode child : getChildrenList(s)) {
       Quota.Counts childCounts = child.cleanSubtree(snapshot, prior,

+ 61 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java

@@ -19,8 +19,10 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Deque;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -674,9 +676,27 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
       if (prior != null) {
         DirectoryDiff priorDiff = this.getDiffs().getDiff(prior);
         if (priorDiff != null) {
-          for (INode cNode : priorDiff.getChildrenDiff().getList(ListType.CREATED)) {
+          // For files/directories created between "prior" and "snapshot", 
+          // we need to clear snapshot copies for "snapshot". Note that we must
+          // use null as prior in the cleanSubtree call. Files/directories that
+          // were created before "prior" will be covered by the later 
+          // cleanSubtreeRecursively call.
+          for (INode cNode : priorDiff.getChildrenDiff().getList(
+              ListType.CREATED)) {
             counts.add(cNode.cleanSubtree(snapshot, null, collectedBlocks));
           }
+          // When a directory is moved from the deleted list of the posterior
+          // diff to the deleted list of this diff, we need to destroy its
+          // descendants that were 1) created after taking this diff and 2)
+          // deleted after taking posterior diff.
+
+          // For files moved from posterior's deleted list, we also need to
+          // delete its snapshot copy associated with the posterior snapshot.
+          for (INode dNode : priorDiff.getChildrenDiff().getList(
+              ListType.DELETED)) {
+            counts.add(cleanDeletedINode(dNode, snapshot, prior,
+                collectedBlocks));
+          }
         }
       }
     }
@@ -688,6 +708,46 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
     return counts;
   }
+  
+  /**
+   * Clean an inode while we move it from the deleted list of post to the
+   * deleted list of prior.
+   * @param inode The inode to clean.
+   * @param post The post snapshot.
+   * @param prior The prior snapshot.
+   * @param collectedBlocks Used to collect blocks for later deletion.
+   * @return Quota usage update.
+   */
+  private Quota.Counts cleanDeletedINode(INode inode, Snapshot post,
+      Snapshot prior, final BlocksMapUpdateInfo collectedBlocks) {
+    Quota.Counts counts = Quota.Counts.newInstance();
+    Deque<INode> queue = new ArrayDeque<INode>();
+    queue.addLast(inode);
+    while (!queue.isEmpty()) {
+      INode topNode = queue.pollFirst();
+      if (topNode instanceof FileWithSnapshot) {
+        FileWithSnapshot fs = (FileWithSnapshot) topNode;
+        counts.add(fs.getDiffs().deleteSnapshotDiff(post, prior,
+            topNode.asFile(), collectedBlocks));
+      } else if (topNode.isDirectory()) {
+        INodeDirectory dir = topNode.asDirectory();
+        if (dir instanceof INodeDirectoryWithSnapshot) {
+          // delete files/dirs created after prior. Note that these
+          // files/dirs, along with inode, were deleted right after post.
+          INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) dir;
+          DirectoryDiff priorDiff = sdir.getDiffs().getDiff(prior);
+          if (priorDiff != null) {
+            counts.add(priorDiff.diff.destroyCreatedList(sdir,
+                collectedBlocks));
+          }
+        }
+        for (INode child : dir.getChildrenList(prior)) {
+          queue.addLast(child);
+        }
+      }
+    }
+    return counts;
+  }
 
   @Override
   public void destroyAndCollectBlocks(

+ 63 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java

@@ -322,11 +322,9 @@ public class TestFSImageWithSnapshot {
     DFSTestUtil.createFile(hdfs, sub1file1, BLOCKSIZE, REPLICATION, seed);
     DFSTestUtil.createFile(hdfs, sub1file2, BLOCKSIZE, REPLICATION, seed);
     
-    // 1. create snapshot s0
     hdfs.allowSnapshot(dir.toString());
     hdfs.createSnapshot(dir, "s0");
     
-    // 2. create snapshot s1 before appending sub1file1 finishes
     HdfsDataOutputStream out = appendFileWithoutClosing(sub1file1, BLOCKSIZE);
     out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));      
     
@@ -347,7 +345,7 @@ public class TestFSImageWithSnapshot {
    * Test fsimage loading when 1) there is an empty file loaded from fsimage,
    * and 2) there is later an append operation to be applied from edit log.
    */
-  @Test
+  @Test (timeout=60000)
   public void testLoadImageWithEmptyFile() throws Exception {
     // create an empty file
     Path file = new Path(dir, "file");
@@ -374,4 +372,66 @@ public class TestFSImageWithSnapshot {
     FileStatus status = hdfs.getFileStatus(file);
     assertEquals(1, status.getLen());
   }
+  
+  /**
+   * Testing a special case with snapshots. When the following steps happen:
+   * <pre>
+   * 1. Take snapshot s1 on dir.
+   * 2. Create new dir and files under subsubDir, which is descendant of dir.
+   * 3. Take snapshot s2 on dir.
+   * 4. Delete subsubDir.
+   * 5. Delete snapshot s2.
+   * </pre>
+   * When we merge the diff from s2 to s1 (since we deleted s2), we need to make
+   * sure all the files/dirs created after s1 should be destroyed. Otherwise
+   * we may save these files/dirs to the fsimage, and cause FileNotFound 
+   * Exception while loading fsimage.  
+   */
+  @Test (timeout=300000)
+  public void testSaveLoadImageAfterSnapshotDeletion()
+      throws Exception {
+    // create initial dir and subdir
+    Path dir = new Path("/dir");
+    Path subDir = new Path(dir, "subdir");
+    Path subsubDir = new Path(subDir, "subsubdir");
+    hdfs.mkdirs(subsubDir);
+    
+    // take snapshots on subdir and dir
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
+    
+    // create new dir under initial dir
+    Path newDir = new Path(subsubDir, "newdir");
+    Path newFile = new Path(newDir, "newfile");
+    hdfs.mkdirs(newDir);
+    DFSTestUtil.createFile(hdfs, newFile, BLOCKSIZE, REPLICATION, seed);
+    
+    // create another snapshot
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s2");
+    
+    // delete subsubdir
+    hdfs.delete(subsubDir, true);
+    
+    // delete snapshot s2
+    hdfs.deleteSnapshot(dir, "s2");
+    
+    // restart cluster
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .format(false).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    
+    // save namespace to fsimage
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.saveNamespace();
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+  }
 }

+ 51 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

@@ -524,6 +524,57 @@ public class TestSnapshotDeletion {
     testCombineSnapshotDiffImpl(sub, "subsub1/subsubsub1/", 3);
   }
   
+  /**
+   * When combine two snapshots, make sure files/directories created after the 
+   * prior snapshot get destroyed.
+   */
+  @Test (timeout=300000)
+  public void testCombineSnapshotDiff3() throws Exception {
+    // create initial dir and subdir
+    Path dir = new Path("/dir");
+    Path subDir1 = new Path(dir, "subdir1");
+    Path subDir2 = new Path(dir, "subdir2");
+    hdfs.mkdirs(subDir2);
+    Path subsubDir = new Path(subDir1, "subsubdir");
+    hdfs.mkdirs(subsubDir);
+    
+    // take snapshots on subdir and dir
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
+    
+    // create new dir under initial dir
+    Path newDir = new Path(subsubDir, "newdir");
+    Path newFile = new Path(newDir, "newfile");
+    DFSTestUtil.createFile(hdfs, newFile, BLOCKSIZE, REPLICATION, seed);
+    Path newFile2 = new Path(subDir2, "newfile");
+    DFSTestUtil.createFile(hdfs, newFile2, BLOCKSIZE, REPLICATION, seed);
+    
+    // create another snapshot
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s2");
+    
+    checkQuotaUsageComputation(dir, 11, BLOCKSIZE * 2 * REPLICATION);
+    
+    // delete subsubdir and subDir2
+    hdfs.delete(subsubDir, true);
+    hdfs.delete(subDir2, true);
+    
+    // add diff of s2 to subDir1, subsubDir, and subDir2
+    checkQuotaUsageComputation(dir, 14, BLOCKSIZE * 2 * REPLICATION);
+    
+    // delete snapshot s2
+    hdfs.deleteSnapshot(dir, "s2");
+    
+    // delete s2 diff in dir, subDir2, and subsubDir. Delete newFile, newDir,
+    // and newFile2. Rename s2 diff to s1 for subDir1 
+    checkQuotaUsageComputation(dir, 8, 0);
+    // Check rename of snapshot diff in subDir1
+    Path subdir1_s1 = SnapshotTestHelper.getSnapshotPath(dir, "s1",
+        subDir1.getName());
+    Path subdir1_s2 = SnapshotTestHelper.getSnapshotPath(dir, "s2",
+        subDir1.getName());
+    assertTrue(hdfs.exists(subdir1_s1));
+    assertFalse(hdfs.exists(subdir1_s2));
+  }
+  
   /**
    * Test snapshot deletion
    * @param snapshotRoot The dir where the snapshots are created