Browse Source

HDFS-4636. Update quota usage when deleting files/dirs that were created after taking the latest snapshot. Contributed by Jing Zhao

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

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

@@ -213,3 +213,6 @@ Branch-2802 Snapshot (Unreleased)
 
   HDFS-4612. Not to use INode.getParent() when generating snapshot diff report.
   (Jing Zhao via szetszwo)
+
+  HDFS-4636. Update quota usage when deleting files/dirs that were created
+  after taking the latest snapshot. (Jing Zhao via szetszwo)

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

@@ -563,7 +563,7 @@ public class FSDirectory implements Closeable {
     final byte[] srcChildName = srcChild.getLocalNameBytes();
     try {
       // remove src
-      final int removedSrc = removeLastINode(srcIIP);
+      final long removedSrc = removeLastINode(srcIIP);
       if (removedSrc == -1) {
         NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
             + "failed to rename " + src + " to " + dst
@@ -730,7 +730,7 @@ public class FSDirectory implements Closeable {
     verifyQuotaForRename(srcIIP.getINodes(), dstIIP.getINodes());
 
     boolean undoRemoveSrc = true;
-    final int removedSrc = removeLastINode(srcIIP);
+    final long removedSrc = removeLastINode(srcIIP);
     if (removedSrc == -1) {
       error = "Failed to rename " + src + " to " + dst
           + " because the source can not be removed";
@@ -1165,25 +1165,33 @@ public class FSDirectory implements Closeable {
     iip.setLastINode(targetNode);
 
     // Remove the node from the namespace
-    final int removed = removeLastINode(iip);
+    long removed = removeLastINode(iip);
     if (removed == -1) {
       return -1;
     }
 
     // set the parent's modification time
-    targetNode.getParent().updateModificationTime(mtime, latestSnapshot);
+    final INodeDirectory parent = targetNode.getParent();
+    parent.updateModificationTime(mtime, latestSnapshot);
     if (removed == 0) {
       return 0;
     }
-
+    
     // collect block
-    final long inodesRemoved = targetNode.cleanSubtree(null, latestSnapshot,
-        collectedBlocks).get(Quota.NAMESPACE);
+    if (!targetNode.isInLatestSnapshot(latestSnapshot)) {
+      targetNode.destroyAndCollectBlocks(collectedBlocks);
+    } else {
+      Quota.Counts counts = targetNode.cleanSubtree(null, latestSnapshot,
+          collectedBlocks);
+      parent.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
+          -counts.get(Quota.DISKSPACE));
+      removed = counts.get(Quota.NAMESPACE);
+    }
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
           + targetNode.getFullPathName() + " is removed");
     }
-    return inodesRemoved;
+    return removed;
   }
   
   /**
@@ -1961,7 +1969,7 @@ public class FSDirectory implements Closeable {
    *          1 for removing a non-reference inode. 
    * @throws NSQuotaExceededException 
    */
-  private int removeLastINode(final INodesInPath iip)
+  private long removeLastINode(final INodesInPath iip)
       throws QuotaExceededException {
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
     final INode last = iip.getLastINode();
@@ -1975,13 +1983,15 @@ public class FSDirectory implements Closeable {
       iip.setINode(-2, last.getParent());
     }
     
-    if (latestSnapshot == null) {
+    if (!last.isInLatestSnapshot(latestSnapshot)) {
       final Quota.Counts counts = last.computeQuotaUsage();
       updateCountNoQuotaCheck(iip, iip.getINodes().length - 1,
           -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
 
       if (INodeReference.tryRemoveReference(last) > 0) {
         return 0;
+      } else {
+        return counts.get(Quota.NAMESPACE);
       }
     }
     return 1;

+ 29 - 20
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java

@@ -101,18 +101,20 @@ public class TestSnapshotBlocksMap {
     }
   }
 
-  INodeFile assertBlockCollection(String path, int numBlocks) throws Exception {
-    final INodeFile file = INodeFile.valueOf(fsdir.getINode(path), path);
+  static INodeFile assertBlockCollection(String path, int numBlocks,
+     final FSDirectory dir, final BlockManager blkManager) throws Exception {
+    final INodeFile file = INodeFile.valueOf(dir.getINode(path), path);
     assertEquals(numBlocks, file.getBlocks().length);
     for(BlockInfo b : file.getBlocks()) {
-      assertBlockCollection(file, b);
+      assertBlockCollection(blkManager, file, b);
     }
     return file;
   }
 
-  void assertBlockCollection(final INodeFile file, final BlockInfo b) { 
-    Assert.assertSame(b, blockmanager.getStoredBlock(b));
-    Assert.assertSame(file, blockmanager.getBlockCollection(b));
+  static void assertBlockCollection(final BlockManager blkManager,
+      final INodeFile file, final BlockInfo b) { 
+    Assert.assertSame(b, blkManager.getStoredBlock(b));
+    Assert.assertSame(file, blkManager.getBlockCollection(b));
     Assert.assertSame(file, b.getBlockCollection());
   }
 
@@ -139,7 +141,8 @@ public class TestSnapshotBlocksMap {
     
     // Normal deletion
     {
-      final INodeFile f2 = assertBlockCollection(file2.toString(), 3);
+      final INodeFile f2 = assertBlockCollection(file2.toString(), 3, fsdir,
+          blockmanager);
       BlockInfo[] blocks = f2.getBlocks();
       hdfs.delete(sub2, true);
       // The INode should have been removed from the blocksMap
@@ -159,21 +162,23 @@ public class TestSnapshotBlocksMap {
 
     // set replication so that the inode should be replaced for snapshots
     {
-      INodeFile f1 = assertBlockCollection(file1.toString(), 2);
+      INodeFile f1 = assertBlockCollection(file1.toString(), 2, fsdir,
+          blockmanager);
       Assert.assertSame(INodeFile.class, f1.getClass());
       hdfs.setReplication(file1, (short)2);
-      f1 = assertBlockCollection(file1.toString(), 2);
+      f1 = assertBlockCollection(file1.toString(), 2, fsdir, blockmanager);
       Assert.assertSame(INodeFileWithSnapshot.class, f1.getClass());
     }
     
     // Check the block information for file0
-    final INodeFile f0 = assertBlockCollection(file0.toString(), 4);
+    final INodeFile f0 = assertBlockCollection(file0.toString(), 4, fsdir,
+        blockmanager);
     BlockInfo[] blocks0 = f0.getBlocks();
     
     // Also check the block information for snapshot of file0
     Path snapshotFile0 = SnapshotTestHelper.getSnapshotPath(sub1, "s0",
         file0.getName());
-    assertBlockCollection(snapshotFile0.toString(), 4);
+    assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
     
     // Delete file0
     hdfs.delete(file0, true);
@@ -181,12 +186,12 @@ public class TestSnapshotBlocksMap {
     for(BlockInfo b : blocks0) {
       assertNotNull(blockmanager.getBlockCollection(b));
     }
-    assertBlockCollection(snapshotFile0.toString(), 4);
+    assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
     
     // Compare the INode in the blocksMap with INodes for snapshots
     String s1f0 = SnapshotTestHelper.getSnapshotPath(sub1, "s1",
         file0.getName()).toString();
-    assertBlockCollection(s1f0, 4);
+    assertBlockCollection(s1f0, 4, fsdir, blockmanager);
     
     // Delete snapshot s1
     hdfs.deleteSnapshot(sub1, "s1");
@@ -195,7 +200,7 @@ public class TestSnapshotBlocksMap {
     for(BlockInfo b : blocks0) {
       assertNotNull(blockmanager.getBlockCollection(b));
     }
-    assertBlockCollection(snapshotFile0.toString(), 4);
+    assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
 
     try {
       INodeFile.valueOf(fsdir.getINode(s1f0), s1f0);
@@ -206,21 +211,25 @@ public class TestSnapshotBlocksMap {
     
     // concat file1, file3 and file5 to file4
     if (runConcatTest) {
-      final INodeFile f1 = assertBlockCollection(file1.toString(), 2);
+      final INodeFile f1 = assertBlockCollection(file1.toString(), 2, fsdir,
+          blockmanager);
       final BlockInfo[] f1blocks = f1.getBlocks();
-      final INodeFile f3 = assertBlockCollection(file3.toString(), 5);
+      final INodeFile f3 = assertBlockCollection(file3.toString(), 5, fsdir,
+          blockmanager);
       final BlockInfo[] f3blocks = f3.getBlocks();
-      final INodeFile f5 = assertBlockCollection(file5.toString(), 7);
+      final INodeFile f5 = assertBlockCollection(file5.toString(), 7, fsdir,
+          blockmanager);
       final BlockInfo[] f5blocks = f5.getBlocks();
-      assertBlockCollection(file4.toString(), 1);
+      assertBlockCollection(file4.toString(), 1, fsdir, blockmanager);
 
       hdfs.concat(file4, new Path[]{file1, file3, file5});
 
-      final INodeFile f4 = assertBlockCollection(file4.toString(), 15);
+      final INodeFile f4 = assertBlockCollection(file4.toString(), 15, fsdir,
+          blockmanager);
       final BlockInfo[] blocks4 = f4.getBlocks();
       for(BlockInfo[] blocks : Arrays.asList(f1blocks, f3blocks, blocks4, f5blocks)) {
         for(BlockInfo b : blocks) {
-          assertBlockCollection(f4, b);
+          assertBlockCollection(blockmanager, f4, b);
         }
       }
       assertAllNull(f1, file1, snapshots);

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

@@ -34,12 +34,15 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 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.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.ipc.RemoteException;
@@ -68,6 +71,7 @@ public class TestSnapshotDeletion {
   protected MiniDFSCluster cluster;
   protected FSNamesystem fsn;
   protected FSDirectory fsdir;
+  protected BlockManager blockmanager;
   protected DistributedFileSystem hdfs;
   
   @Rule
@@ -82,6 +86,7 @@ public class TestSnapshotDeletion {
 
     fsn = cluster.getNamesystem();
     fsdir = fsn.getFSDirectory();
+    blockmanager = fsn.getBlockManager();
     hdfs = cluster.getFileSystem();
   }
 
@@ -152,6 +157,12 @@ public class TestSnapshotDeletion {
         dirNode.getNamespace());
     assertEquals(dirNode.dumpTreeRecursively().toString(), expectedDs,
         dirNode.getDiskspace());
+    Quota.Counts counts = Quota.Counts.newInstance();
+    dirNode.computeQuotaUsage(counts, false);
+    assertEquals(dirNode.dumpTreeRecursively().toString(), expectedNs,
+        counts.get(Quota.NAMESPACE));
+    assertEquals(dirNode.dumpTreeRecursively().toString(), expectedDs,
+        counts.get(Quota.DISKSPACE));
   }
   
   /**
@@ -190,8 +201,22 @@ public class TestSnapshotDeletion {
     
     // create snapshot s0
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
+    
+    // after creating snapshot s0, create a directory tempdir under dir and then
+    // delete dir immediately
+    Path tempDir = new Path(dir, "tempdir");
+    Path tempFile = new Path(tempDir, "tempfile");
+    DFSTestUtil.createFile(hdfs, tempFile, BLOCKSIZE, REPLICATION, seed);
+    final INodeFile temp = TestSnapshotBlocksMap.assertBlockCollection(
+        tempFile.toString(), 1, fsdir, blockmanager);
+    BlockInfo[] blocks = temp.getBlocks();
+    hdfs.delete(tempDir, true);
     // check dir's quota usage
     checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 3);
+    // check blocks of tempFile
+    for (BlockInfo b : blocks) {
+      assertNull(blockmanager.getBlockCollection(b));
+    }
     
     // make a change: create a new file under subsub
     Path newFileAfterS0 = new Path(subsub, "newFile");
@@ -235,11 +260,17 @@ public class TestSnapshotDeletion {
     INode noChangeFileSCopy = children.get(1);
     assertEquals(noChangeFile.getName(), noChangeFileSCopy.getLocalName());
     assertEquals(INodeFile.class, noChangeFileSCopy.getClass());
+    TestSnapshotBlocksMap.assertBlockCollection(new Path(snapshotNoChangeDir,
+        noChangeFileSCopy.getLocalName()).toString(), 1, fsdir, blockmanager);
+    
     INodeFileWithSnapshot metaChangeFile2SCopy = 
         (INodeFileWithSnapshot) children.get(0);
     assertEquals(metaChangeFile2.getName(), metaChangeFile2SCopy.getLocalName());
     assertEquals(INodeFileWithSnapshot.class, metaChangeFile2SCopy.getClass());
-    // check the replication factor of metaChangeFile1SCopy
+    TestSnapshotBlocksMap.assertBlockCollection(new Path(snapshotNoChangeDir,
+        metaChangeFile2SCopy.getLocalName()).toString(), 1, fsdir, blockmanager);
+    
+    // check the replication factor of metaChangeFile2SCopy
     assertEquals(REPLICATION_1,
         metaChangeFile2SCopy.getFileReplication(null));
     assertEquals(REPLICATION_1,
@@ -251,10 +282,17 @@ public class TestSnapshotDeletion {
     // before deleting sub, we first create a new file under sub
     Path newFile = new Path(sub, "newFile");
     DFSTestUtil.createFile(hdfs, newFile, BLOCKSIZE, REPLICATION, seed);
+    final INodeFile newFileNode = TestSnapshotBlocksMap.assertBlockCollection(
+        newFile.toString(), 1, fsdir, blockmanager);
+    blocks = newFileNode.getBlocks();
     checkQuotaUsageComputation(dir, 18L, BLOCKSIZE * REPLICATION * 5);
     hdfs.delete(sub, true);
-    // while deletion, we add diff for dir and metaChangeFile1
-    checkQuotaUsageComputation(dir, 20L, BLOCKSIZE * REPLICATION * 5);
+    // while deletion, we add diff for subsub and metaChangeFile1, and remove
+    // newFile
+    checkQuotaUsageComputation(dir, 19L, BLOCKSIZE * REPLICATION * 4);
+    for (BlockInfo b : blocks) {
+      assertNull(blockmanager.getBlockCollection(b));
+    }
     
     // make sure the whole subtree of sub is stored correctly in snapshot
     Path snapshotSub = SnapshotTestHelper.getSnapshotPath(dir, "s1",
@@ -385,6 +423,10 @@ public class TestSnapshotDeletion {
     DFSTestUtil.createFile(hdfs, metaChangeFile, BLOCKSIZE, REPLICATION, seed);
     DFSTestUtil.createFile(hdfs, toDeleteFile, BLOCKSIZE, REPLICATION, seed);
     
+    final INodeFile toDeleteFileNode = TestSnapshotBlocksMap
+        .assertBlockCollection(toDeleteFile.toString(), 1, fsdir, blockmanager);
+    BlockInfo[] blocks = toDeleteFileNode.getBlocks();
+    
     // create snapshot s0 on dir
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
     checkQuotaUsageComputation(dir, 8, 3 * BLOCKSIZE * REPLICATION);
@@ -409,6 +451,9 @@ public class TestSnapshotDeletion {
     // metaChangeDir's diff, dir's diff. diskspace: remove toDeleteFile, and 
     // metaChangeFile's replication factor decreases
     checkQuotaUsageComputation(dir, 7, 2 * BLOCKSIZE * REPLICATION - BLOCKSIZE);
+    for (BlockInfo b : blocks) {
+      assertNull(blockmanager.getBlockCollection(b));
+    }
     
     // check 1. there is no snapshot s0
     final INodeDirectorySnapshottable dirNode = 
@@ -429,6 +474,8 @@ public class TestSnapshotDeletion {
     final INodeFile noChangeFileNode = 
         (INodeFile) fsdir.getINode(noChangeFile.toString());
     assertEquals(INodeFile.class, noChangeFileNode.getClass());
+    TestSnapshotBlocksMap.assertBlockCollection(noChangeFile.toString(), 1,
+        fsdir, blockmanager);
     
     // check 3: current metadata of metaChangeFile and metaChangeDir
     FileStatus status = hdfs.getFileStatus(metaChangeDir);
@@ -436,6 +483,8 @@ public class TestSnapshotDeletion {
     assertEquals("unknown", status.getGroup());
     status = hdfs.getFileStatus(metaChangeFile);
     assertEquals(REPLICATION_1, status.getReplication());
+    TestSnapshotBlocksMap.assertBlockCollection(metaChangeFile.toString(), 1,
+        fsdir, blockmanager);
     
     // check 4: no snapshot copy for toDeleteFile
     try {
@@ -575,10 +624,19 @@ public class TestSnapshotDeletion {
         modDirStr + "file12");
     Path file13_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
         modDirStr + "file13");
+    Path file14_s2 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s2",
+        modDirStr + "file14");
+    Path file15_s2 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s2",
+        modDirStr + "file15");
     FileStatus statusBeforeDeletion10 = hdfs.getFileStatus(file10_s1);
     FileStatus statusBeforeDeletion11 = hdfs.getFileStatus(file11_s1);
     FileStatus statusBeforeDeletion12 = hdfs.getFileStatus(file12_s1);
     FileStatus statusBeforeDeletion13 = hdfs.getFileStatus(file13_s1);
+    INodeFile file14Node = TestSnapshotBlocksMap.assertBlockCollection(
+        file14_s2.toString(), 1, fsdir, blockmanager);
+    BlockInfo[] blocks_14 = file14Node.getBlocks();
+    TestSnapshotBlocksMap.assertBlockCollection(file15_s2.toString(), 1, fsdir,
+        blockmanager);
     
     // delete s2, in which process we need to combine the diff in s2 to s1
     hdfs.deleteSnapshot(snapshotRoot, "s2");
@@ -598,18 +656,31 @@ public class TestSnapshotDeletion {
         statusAfterDeletion12.toString());
     assertEquals(statusBeforeDeletion13.toString(),
         statusAfterDeletion13.toString());
+    TestSnapshotBlocksMap.assertBlockCollection(file10_s1.toString(), 1, fsdir,
+        blockmanager);
+    TestSnapshotBlocksMap.assertBlockCollection(file11_s1.toString(), 1, fsdir,
+        blockmanager);
+    TestSnapshotBlocksMap.assertBlockCollection(file12_s1.toString(), 1, fsdir,
+        blockmanager);
+    TestSnapshotBlocksMap.assertBlockCollection(file13_s1.toString(), 1, fsdir,
+        blockmanager);
     
     // make sure file14 and file15 are not included in s1
     Path file14_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
         modDirStr + "file14");
-    assertFalse(hdfs.exists(file14_s1));
     Path file15_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
         modDirStr + "file15");
+    assertFalse(hdfs.exists(file14_s1));
     assertFalse(hdfs.exists(file15_s1));
+    for (BlockInfo b : blocks_14) {
+      assertNull(blockmanager.getBlockCollection(b));
+    }
     
     INodeFile nodeFile13 = (INodeFile) fsdir.getINode(file13.toString());
     assertEquals(REPLICATION_1, nodeFile13.getBlockReplication());
-
+    TestSnapshotBlocksMap.assertBlockCollection(file13.toString(), 1, fsdir,
+        blockmanager);
+    
     INodeFile nodeFile12 = (INodeFile) fsdir.getINode(file12_s1.toString());
     assertEquals(REPLICATION_1, nodeFile12.getBlockReplication());
   }