Browse Source

HDFS-4809. When a QuotaExceededException is thrown during rename, the quota usage should be subtracted back. Contributed by Jing Zhao

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

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

@@ -351,3 +351,6 @@ Branch-2802 Snapshot (Unreleased)
   HDFS-4806. In INodeDirectoryWithSnapshot, use isInLatestSnapshot() to 
   HDFS-4806. In INodeDirectoryWithSnapshot, use isInLatestSnapshot() to 
   determine if an added/removed child should be recorded in the snapshot diff.
   determine if an added/removed child should be recorded in the snapshot diff.
   (Jing Zhao via szetszwo)
   (Jing Zhao via szetszwo)
+
+  HDFS-4809. When a QuotaExceededException is thrown during rename, the quota
+  usage should be subtracted back.  (Jing Zhao via szetszwo)

+ 10 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -710,7 +710,7 @@ public class FSDirectory implements Closeable {
           ((INodeDirectoryWithSnapshot) srcParent).undoRename4ScrParent(
           ((INodeDirectoryWithSnapshot) srcParent).undoRename4ScrParent(
               oldSrcChild.asReference(), srcChild, srcIIP.getLatestSnapshot());
               oldSrcChild.asReference(), srcChild, srcIIP.getLatestSnapshot());
         } else {
         } else {
-          // srcParent is not an INodeDirectoryWithSnapshot, we only need to add
+          // original srcChild is not in latest snapshot, we only need to add
           // the srcChild back
           // the srcChild back
           addLastINodeNoQuotaCheck(srcIIP, srcChild);
           addLastINodeNoQuotaCheck(srcIIP, srcChild);
         }
         }
@@ -2185,8 +2185,15 @@ public class FSDirectory implements Closeable {
     updateCount(iip, pos,
     updateCount(iip, pos,
         counts.get(Quota.NAMESPACE), counts.get(Quota.DISKSPACE), checkQuota);
         counts.get(Quota.NAMESPACE), counts.get(Quota.DISKSPACE), checkQuota);
     final INodeDirectory parent = inodes[pos-1].asDirectory();
     final INodeDirectory parent = inodes[pos-1].asDirectory();
-    final boolean added = parent.addChild(child, true, iip.getLatestSnapshot(),
-        inodeMap);
+    boolean added = false;
+    try {
+      added = parent.addChild(child, true, iip.getLatestSnapshot(),
+          inodeMap);
+    } catch (QuotaExceededException e) {
+      updateCountNoQuotaCheck(iip, pos,
+          -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
+      throw e;
+    }
     if (!added) {
     if (!added) {
       updateCountNoQuotaCheck(iip, pos,
       updateCountNoQuotaCheck(iip, pos,
           -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
           -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));

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

@@ -855,7 +855,6 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   
   
   /**
   /**
    * Destroy a subtree under a DstReference node.
    * Destroy a subtree under a DstReference node.
-   * @see INodeReference.DstReference#destroyAndCollectBlocks(BlocksMapUpdateInfo, List)
    */
    */
   public static void destroyDstSubtree(INode inode, final Snapshot snapshot,
   public static void destroyDstSubtree(INode inode, final Snapshot snapshot,
       final Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
       final Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,

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

@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
@@ -56,6 +57,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.ChildrenDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.ChildrenDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
@@ -406,7 +408,7 @@ public class TestRenameWithSnapshots {
    * Test renaming a dir and then delete snapshots.
    * Test renaming a dir and then delete snapshots.
    */
    */
   @Test
   @Test
-  public void testRenameDirAndDeleteSnapshot() throws Exception {
+  public void testRenameDirAndDeleteSnapshot_1() throws Exception {
     final Path sdir1 = new Path("/dir1");
     final Path sdir1 = new Path("/dir1");
     final Path sdir2 = new Path("/dir2");
     final Path sdir2 = new Path("/dir2");
     hdfs.mkdirs(sdir1);
     hdfs.mkdirs(sdir1);
@@ -1237,7 +1239,7 @@ public class TestRenameWithSnapshots {
    * file/dir before taking the snapshot.
    * file/dir before taking the snapshot.
    */
    */
   @Test
   @Test
-  public void testRenameUndo() throws Exception {
+  public void testRenameUndo_1() throws Exception {
     final Path sdir1 = new Path("/dir1");
     final Path sdir1 = new Path("/dir1");
     final Path sdir2 = new Path("/dir2");
     final Path sdir2 = new Path("/dir2");
     hdfs.mkdirs(sdir1);
     hdfs.mkdirs(sdir1);
@@ -1510,6 +1512,197 @@ public class TestRenameWithSnapshots {
     assertSame(foo3Node, foo3_wc.getParentReference());
     assertSame(foo3Node, foo3_wc.getParentReference());
   }
   }
   
   
+  /**
+   * Test rename while the rename operation will exceed the quota in the dst
+   * tree.
+   */
+  @Test
+  public void testRenameUndo_5() throws Exception {
+    final Path test = new Path("/test");
+    final Path dir1 = new Path(test, "dir1");
+    final Path dir2 = new Path(test, "dir2");
+    final Path subdir2 = new Path(dir2, "subdir2");
+    hdfs.mkdirs(dir1);
+    hdfs.mkdirs(subdir2);
+    
+    final Path foo = new Path(dir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
+    
+    // set ns quota of dir2 to 5, so the current remaining is 2 (already has
+    // dir2, subdir2, and s2)
+    hdfs.setQuota(dir2, 5, Long.MAX_VALUE - 1);
+    
+    final Path foo2 = new Path(subdir2, foo.getName());
+    // rename /test/dir1/foo to /test/dir2/subdir2/foo. 
+    // FSDirectory#verifyQuota4Rename will pass since foo/bar only be counted 
+    // as 2 in NS quota. However, the rename operation will fail when adding
+    // foo to subdir2, since we will create a snapshot diff for subdir2. 
+    boolean rename = hdfs.rename(foo, foo2);
+    assertFalse(rename);
+    
+    // check the undo
+    assertTrue(hdfs.exists(foo));
+    assertTrue(hdfs.exists(bar));
+    INodeDirectory dir1Node = fsdir.getINode4Write(dir1.toString())
+        .asDirectory();
+    List<INode> childrenList = ReadOnlyList.Util.asList(dir1Node
+        .getChildrenList(null));
+    assertEquals(1, childrenList.size());
+    INode fooNode = childrenList.get(0);
+    assertTrue(fooNode.getClass() == INodeDirectoryWithSnapshot.class);
+    INode barNode = fsdir.getINode4Write(bar.toString());
+    assertTrue(barNode.getClass() == INodeFile.class);
+    assertSame(fooNode, barNode.getParent());
+    List<DirectoryDiff> diffList = ((INodeDirectorySnapshottable) dir1Node)
+        .getDiffs().asList();
+    assertEquals(1, diffList.size());
+    DirectoryDiff diff = diffList.get(0);
+    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
+    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
+    
+    // check dir2
+    INode dir2Node = fsdir.getINode4Write(dir2.toString());
+    assertTrue(dir2Node.getClass() == INodeDirectorySnapshottable.class);
+    Quota.Counts counts = dir2Node.computeQuotaUsage();
+    assertEquals(3, counts.get(Quota.NAMESPACE));
+    assertEquals(0, counts.get(Quota.DISKSPACE));
+    childrenList = ReadOnlyList.Util.asList(dir2Node.asDirectory()
+        .getChildrenList(null));
+    assertEquals(1, childrenList.size());
+    INode subdir2Node = childrenList.get(0);
+    assertSame(dir2Node, subdir2Node.getParent());
+    assertSame(subdir2Node, fsdir.getINode4Write(subdir2.toString()));
+    diffList = ((INodeDirectorySnapshottable) dir2Node)
+        .getDiffs().asList();
+    assertEquals(1, diffList.size());
+    diff = diffList.get(0);
+    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
+    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
+  }
+  
+  /**
+   * Test the rename undo when removing dst node fails
+   */
+  @Test
+  public void testRenameUndo_6() throws Exception {
+    final Path test = new Path("/test");
+    final Path dir1 = new Path(test, "dir1");
+    final Path dir2 = new Path(test, "dir2");
+    final Path sub_dir2 = new Path(dir2, "subdir");
+    final Path subsub_dir2 = new Path(sub_dir2, "subdir");
+    hdfs.mkdirs(dir1);
+    hdfs.mkdirs(subsub_dir2);
+    
+    final Path foo = new Path(dir1, "foo");
+    hdfs.mkdirs(foo);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
+    
+    // set ns quota of dir2 to 4, so the current remaining is 0 (already has
+    // dir2, sub_dir2, subsub_dir2, and s2)
+    hdfs.setQuota(dir2, 4, Long.MAX_VALUE - 1);
+    
+    // rename /test/dir1/foo to /test/dir2/sub_dir2/subsub_dir2. 
+    // FSDirectory#verifyQuota4Rename will pass since foo only be counted 
+    // as 1 in NS quota. However, the rename operation will fail when removing
+    // subsub_dir2 since this step tries to add a snapshot diff in sub_dir2.
+    try {
+      hdfs.rename(foo, subsub_dir2, Rename.OVERWRITE);
+      fail("Expect QuotaExceedException");
+    } catch (QuotaExceededException e) {
+      String msg = "Failed to record modification for snapshot: "
+          + "The NameSpace quota (directories and files)"
+          + " is exceeded: quota=4 file count=5"; 
+      GenericTestUtils.assertExceptionContains(msg, e);
+    }
+    
+    // check the undo
+    assertTrue(hdfs.exists(foo));
+    INodeDirectory dir1Node = fsdir.getINode4Write(dir1.toString())
+        .asDirectory();
+    List<INode> childrenList = ReadOnlyList.Util.asList(dir1Node
+        .getChildrenList(null));
+    assertEquals(1, childrenList.size());
+    INode fooNode = childrenList.get(0);
+    assertTrue(fooNode.getClass() == INodeDirectoryWithSnapshot.class);
+    assertSame(dir1Node, fooNode.getParent());
+    List<DirectoryDiff> diffList = ((INodeDirectorySnapshottable) dir1Node)
+        .getDiffs().asList();
+    assertEquals(1, diffList.size());
+    DirectoryDiff diff = diffList.get(0);
+    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
+    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
+    
+    // check dir2
+    INode dir2Node = fsdir.getINode4Write(dir2.toString());
+    assertTrue(dir2Node.getClass() == INodeDirectorySnapshottable.class);
+    Quota.Counts counts = dir2Node.computeQuotaUsage();
+    assertEquals(4, counts.get(Quota.NAMESPACE));
+    assertEquals(0, counts.get(Quota.DISKSPACE));
+    childrenList = ReadOnlyList.Util.asList(dir2Node.asDirectory()
+        .getChildrenList(null));
+    assertEquals(1, childrenList.size());
+    INode subdir2Node = childrenList.get(0);
+    assertTrue(subdir2Node.getClass() == INodeDirectoryWithSnapshot.class);
+    assertSame(dir2Node, subdir2Node.getParent());
+    assertSame(subdir2Node, fsdir.getINode4Write(sub_dir2.toString()));
+    INode subsubdir2Node = fsdir.getINode4Write(subsub_dir2.toString());
+    assertTrue(subsubdir2Node.getClass() == INodeDirectory.class);
+    assertSame(subdir2Node, subsubdir2Node.getParent());
+    
+    diffList = ((INodeDirectorySnapshottable) dir2Node).getDiffs().asList();
+    assertEquals(1, diffList.size());
+    diff = diffList.get(0);
+    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
+    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
+    
+    diffList = ((INodeDirectoryWithSnapshot) subdir2Node).getDiffs().asList();
+    assertEquals(0, diffList.size());
+  }
+  
+  /**
+   * Test the rename undo when quota of dst tree is exceeded after rename.
+   */
+  @Test
+  public void testRenameExceedQuota() throws Exception {
+    final Path test = new Path("/test");
+    final Path dir1 = new Path(test, "dir1");
+    final Path dir2 = new Path(test, "dir2");
+    final Path sub_dir2 = new Path(dir2, "subdir");
+    final Path subfile_dir2 = new Path(sub_dir2, "subfile");
+    hdfs.mkdirs(dir1);
+    DFSTestUtil.createFile(hdfs, subfile_dir2, BLOCKSIZE, REPL, SEED);
+    
+    final Path foo = new Path(dir1, "foo");
+    DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
+    
+    // set ns quota of dir2 to 4, so the current remaining is 1 (already has
+    // dir2, sub_dir2, subfile_dir2, and s2)
+    hdfs.setQuota(dir2, 5, Long.MAX_VALUE - 1);
+    
+    // rename /test/dir1/foo to /test/dir2/sub_dir2/subfile_dir2. 
+    // FSDirectory#verifyQuota4Rename will pass since foo only be counted 
+    // as 1 in NS quota. The rename operation will succeed while the real quota 
+    // of dir2 will become 7 (dir2, s2 in dir2, sub_dir2, s2 in sub_dir2,
+    // subfile_dir2 in deleted list, new subfile, s1 in new subfile).
+    hdfs.rename(foo, subfile_dir2, Rename.OVERWRITE);
+    
+    // check dir2
+    INode dir2Node = fsdir.getINode4Write(dir2.toString());
+    assertTrue(dir2Node.getClass() == INodeDirectorySnapshottable.class);
+    Quota.Counts counts = dir2Node.computeQuotaUsage();
+    assertEquals(7, counts.get(Quota.NAMESPACE));
+    assertEquals(BLOCKSIZE * REPL * 2, counts.get(Quota.DISKSPACE));
+  }
+  
   @Test
   @Test
   public void testRename2PreDescendant() throws Exception {
   public void testRename2PreDescendant() throws Exception {
     final Path sdir1 = new Path("/dir1");
     final Path sdir1 = new Path("/dir1");
@@ -1685,7 +1878,7 @@ public class TestRenameWithSnapshots {
    * added/removed child should be recorded in snapshots.
    * added/removed child should be recorded in snapshots.
    */
    */
   @Test
   @Test
-  public void testRenameAndDeleteSnapshot_5() throws Exception {
+  public void testRenameDirAndDeleteSnapshot_5() throws Exception {
     final Path dir1 = new Path("/dir1");
     final Path dir1 = new Path("/dir1");
     final Path dir2 = new Path("/dir2");
     final Path dir2 = new Path("/dir2");
     final Path dir3 = new Path("/dir3");
     final Path dir3 = new Path("/dir3");