Browse Source

HDFS-4650. Fix a bug in FSDirectory and add more unit tests for rename with existence of snapshottable directories and snapshots. Contributed by Jing Zhao

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1476012 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 years ago
parent
commit
9da3865841

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

@@ -308,3 +308,7 @@ Branch-2802 Snapshot (Unreleased)
 
   HDFS-4755. Fix AccessControlException message and moves "implements
   LinkedElement" from INode to INodeWithAdditionalFields.  (szetszwo)
+
+  HDFS-4650. Fix a bug in FSDirectory and add more unit tests for rename with
+  existence of snapshottable directories and snapshots.  (Jing Zhao via
+  szetszwo)

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

@@ -677,6 +677,7 @@ public class FSDirectory implements Closeable {
         // update modification time of dst and the parent of src
         final INode srcParent = srcIIP.getINode(-2);
         srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot());
+        dstParent = dstIIP.getINode(-2); // refresh dstParent
         dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot());
         // update moved leases with new filename
         getFSNamesystem().unprotectedChangeLease(src, dst);     

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

@@ -1528,4 +1528,148 @@ public class TestRenameWithSnapshots {
     
     restartClusterAndCheckImage();
   }
+  
+  /**
+   * After the following operations:
+   * Rename a dir -> create a snapshot s on dst tree -> delete the renamed dir
+   * -> delete snapshot s on dst tree
+   * 
+   * Make sure we destroy everything created after the rename under the renamed
+   * dir.
+   */
+  @Test
+  public void testRenameDirAndDeleteSnapshot_3() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    hdfs.mkdirs(sdir2);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    
+    final Path foo2 = new Path(sdir2, "foo");
+    hdfs.rename(foo, foo2);
+    
+    // create two new files under foo2
+    final Path bar2 = new Path(foo2, "bar2");
+    DFSTestUtil.createFile(hdfs, bar2, BLOCKSIZE, REPL, SEED);
+    final Path bar3 = new Path(foo2, "bar3");
+    DFSTestUtil.createFile(hdfs, bar3, BLOCKSIZE, REPL, SEED);
+    
+    // create a new snapshot on sdir2
+    hdfs.createSnapshot(sdir2, "s3");
+    
+    // delete foo2
+    hdfs.delete(foo2, true);
+    // delete s3
+    hdfs.deleteSnapshot(sdir2, "s3");
+    
+    // check
+    final INodeDirectorySnapshottable dir1Node = 
+        (INodeDirectorySnapshottable) fsdir.getINode4Write(sdir1.toString());
+    assertEquals(4, dir1Node.getNamespace());
+    final INodeDirectorySnapshottable dir2Node = 
+        (INodeDirectorySnapshottable) fsdir.getINode4Write(sdir2.toString());
+    assertEquals(2, dir2Node.getNamespace());
+    
+    final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
+        foo.getName());
+    INode fooRef = fsdir.getINode(foo_s1.toString());
+    assertTrue(fooRef instanceof INodeReference.WithName);
+    INodeReference.WithCount wc = 
+        (WithCount) fooRef.asReference().getReferredINode();
+    assertEquals(1, wc.getReferenceCount());
+    INodeDirectoryWithSnapshot fooNode = 
+        (INodeDirectoryWithSnapshot) wc.getReferredINode().asDirectory();
+    ReadOnlyList<INode> children = fooNode.getChildrenList(null);
+    assertEquals(1, children.size());
+    assertEquals(bar.getName(), children.get(0).getLocalName());
+    List<DirectoryDiff> diffList = fooNode.getDiffs().asList();
+    assertEquals(1, diffList.size());
+    assertEquals("s1", Snapshot.getSnapshotName(diffList.get(0).snapshot));
+    ChildrenDiff diff = diffList.get(0).getChildrenDiff();
+    assertEquals(0, diff.getList(ListType.CREATED).size());
+    assertEquals(0, diff.getList(ListType.DELETED).size());
+    
+    restartClusterAndCheckImage();
+  }
+  
+  /**
+   * After the following operations:
+   * Rename a dir -> create a snapshot s on dst tree -> rename the renamed dir
+   * again -> delete snapshot s on dst tree
+   * 
+   * Make sure we only delete the snapshot s under the renamed dir.
+   */
+  @Test
+  public void testRenameDirAndDeleteSnapshot_4() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    hdfs.mkdirs(sdir2);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    
+    final Path foo2 = new Path(sdir2, "foo");
+    hdfs.rename(foo, foo2);
+    
+    // create two new files under foo2
+    final Path bar2 = new Path(foo2, "bar2");
+    DFSTestUtil.createFile(hdfs, bar2, BLOCKSIZE, REPL, SEED);
+    final Path bar3 = new Path(foo2, "bar3");
+    DFSTestUtil.createFile(hdfs, bar3, BLOCKSIZE, REPL, SEED);
+    
+    // create a new snapshot on sdir2
+    hdfs.createSnapshot(sdir2, "s3");
+    
+    // rename foo2 again
+    hdfs.rename(foo2, foo);
+    // delete snapshot s3
+    hdfs.deleteSnapshot(sdir2, "s3");
+    
+    // check
+    final INodeDirectorySnapshottable dir1Node = 
+        (INodeDirectorySnapshottable) fsdir.getINode4Write(sdir1.toString());
+    // sdir1 + s1 + foo_s1 (foo) + foo (foo + s1 + bar~bar3)
+    assertEquals(9, dir1Node.getNamespace());
+    final INodeDirectorySnapshottable dir2Node = 
+        (INodeDirectorySnapshottable) fsdir.getINode4Write(sdir2.toString());
+    assertEquals(2, dir2Node.getNamespace());
+    
+    final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
+        foo.getName());
+    final INode fooRef = fsdir.getINode(foo_s1.toString());
+    assertTrue(fooRef instanceof INodeReference.WithName);
+    INodeReference.WithCount wc = 
+        (WithCount) fooRef.asReference().getReferredINode();
+    assertEquals(2, wc.getReferenceCount());
+    INodeDirectoryWithSnapshot fooNode = 
+        (INodeDirectoryWithSnapshot) wc.getReferredINode().asDirectory();
+    ReadOnlyList<INode> children = fooNode.getChildrenList(null);
+    assertEquals(3, children.size());
+    assertEquals(bar.getName(), children.get(0).getLocalName());
+    assertEquals(bar2.getName(), children.get(1).getLocalName());
+    assertEquals(bar3.getName(), children.get(2).getLocalName());
+    List<DirectoryDiff> diffList = fooNode.getDiffs().asList();
+    assertEquals(1, diffList.size());
+    assertEquals("s1", Snapshot.getSnapshotName(diffList.get(0).snapshot));
+    ChildrenDiff diff = diffList.get(0).getChildrenDiff();
+    // bar2 and bar3 in the created list
+    assertEquals(2, diff.getList(ListType.CREATED).size());
+    assertEquals(0, diff.getList(ListType.DELETED).size());
+    
+    final INode fooRef2 = fsdir.getINode4Write(foo.toString());
+    assertTrue(fooRef2 instanceof INodeReference.DstReference);
+    INodeReference.WithCount wc2 = 
+        (WithCount) fooRef2.asReference().getReferredINode();
+    assertSame(wc, wc2);
+    assertSame(fooRef2, wc.getParentReference());
+    
+    restartClusterAndCheckImage();
+  }
 }

+ 88 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper.TestDirectoryTree;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper.TestDirectoryTree.Node;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
@@ -216,7 +217,7 @@ public class TestSnapshot {
    * -> Check previous snapshots -----------------+
    * </pre>
    */
-  @Test (timeout=300000)
+  @Test
   public void testSnapshot() throws Throwable {
     try {
       runTestSnapshot();
@@ -381,6 +382,10 @@ public class TestSnapshot {
       node.nullFileIndex = (node.nullFileIndex + 1) % node.fileList.size();
       Modification dirChange = new DirCreationOrDeletion(node.nodePath, hdfs,
           node, random.nextBoolean());
+      // dir rename
+      Node dstParent = dirTree.getRandomDirNode(random, Arrays.asList(nodes));
+      Modification dirRename = new DirRename(node.nodePath, hdfs, node,
+          dstParent);
       
       mList.add(create);
       mList.add(delete);
@@ -391,8 +396,9 @@ public class TestSnapshot {
       mList.add(chown);
       mList.add(replication);
       mList.add(dirChange);
+      mList.add(dirRename);
     }
-    return mList.toArray(new Modification[0]);
+    return mList.toArray(new Modification[mList.size()]);
   }
   
   /**
@@ -850,5 +856,84 @@ public class TestSnapshot {
         }
       }
     } 
-  } 
+  }
+  
+  /**
+   * Directory creation or deletion.
+   */
+  class DirRename extends Modification {
+    private final TestDirectoryTree.Node srcParent;
+    private final TestDirectoryTree.Node dstParent;
+    private final Path srcPath;
+    private final Path dstPath;
+    private final HashMap<Path, FileStatus> statusMap;
+    
+    DirRename(Path file, FileSystem fs, TestDirectoryTree.Node src,
+        TestDirectoryTree.Node dst) throws Exception {
+      super(file, fs, "dirrename");
+      this.srcParent = src;
+      this.dstParent = dst;
+      dstPath = new Path(dstParent.nodePath, "sub"
+          + dstParent.nonSnapshotChildren.size());
+      
+      // If the srcParent's nonSnapshotChildren is empty, we need to create
+      // sub-directories
+      if (srcParent.nonSnapshotChildren.isEmpty()) {
+        srcPath = new Path(srcParent.nodePath, "sub"
+            + srcParent.nonSnapshotChildren.size());
+        // creation
+        TestDirectoryTree.Node newChild = new TestDirectoryTree.Node(
+            srcPath, srcParent.level + 1, srcParent, hdfs);
+        // create file under the new non-snapshottable directory
+        newChild.initFileList(hdfs, srcParent.nodePath.getName(), BLOCKSIZE,
+            REPLICATION, seed, 2);
+        srcParent.nonSnapshotChildren.add(newChild);
+      } else {
+        srcPath = new Path(srcParent.nodePath, "sub"
+            + (srcParent.nonSnapshotChildren.size() - 1));
+      }
+      this.statusMap = new HashMap<Path, FileStatus>();
+    }
+    
+    @Override
+    void loadSnapshots() throws Exception {
+      for (Path snapshotRoot : snapshotList) {
+        Path snapshotDir = SnapshotTestHelper.getSnapshotFile(snapshotRoot,
+            srcPath);
+        if (snapshotDir != null) {
+          FileStatus status = fs.exists(snapshotDir) ? fs
+              .getFileStatus(snapshotDir) : null;
+          statusMap.put(snapshotDir, status);
+          // In each non-snapshottable directory, we also create a file. Thus
+          // here we also need to check the file's status before/after taking
+          // snapshots
+          Path snapshotFile = new Path(snapshotDir, "file0");
+          status = fs.exists(snapshotFile) ? fs.getFileStatus(snapshotFile)
+              : null;
+          statusMap.put(snapshotFile, status);
+        }
+      }
+    }
+
+    @Override
+    void modify() throws Exception {
+      hdfs.rename(srcPath, dstPath);
+      TestDirectoryTree.Node newDstChild = new TestDirectoryTree.Node(
+          dstPath, dstParent.level + 1, dstParent, hdfs);
+      dstParent.nonSnapshotChildren.add(newDstChild);
+    }
+
+    @Override
+    void checkSnapshots() throws Exception {
+      for (Path snapshot : statusMap.keySet()) {
+        FileStatus currentStatus = fs.exists(snapshot) ? fs
+            .getFileStatus(snapshot) : null;
+        FileStatus originalStatus = statusMap.get(snapshot);
+        assertEquals(currentStatus, originalStatus);
+        if (currentStatus != null) {
+          assertEquals(currentStatus.toString(), originalStatus.toString());
+        }
+      }
+    } 
+  }
 }