Browse Source

HDFS-7728. Avoid updating quota usage while loading edits. Contributed by Jing Zhao.

Haohui Mai 10 năm trước cách đây
mục cha
commit
4b5915be9f
27 tập tin đã thay đổi với 465 bổ sung417 xóa
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 26 35
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
  3. 10 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
  4. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
  5. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  6. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  7. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  8. 119 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  9. 39 42
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  10. 41 42
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  11. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
  12. 55 63
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  13. 7 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
  14. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/QuotaCounts.java
  15. 3 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
  16. 9 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  17. 1 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
  18. 48 69
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
  19. 10 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
  20. 13 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
  21. 15 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
  22. 4 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
  23. 10 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaByStorageType.java
  24. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
  25. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  26. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  27. 1 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java

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

@@ -425,6 +425,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8243. Files written by TestHostsFiles and TestNameNodeMXBean are
     causing Release Audit Warnings. (Ruth Wisniewski via Arpit Agarwal)
 
+    HDFS-7728. Avoid updating quota usage while loading edits.
+    (Jing Zhao via wheat9)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 26 - 35
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+import org.apache.hadoop.hdfs.server.namenode.INode.ReclaimContext;
 import org.apache.hadoop.util.ChunkedArrayList;
 
 import java.io.IOException;
@@ -39,24 +40,26 @@ class FSDirDeleteOp {
    * @param removedINodes INodes that should be removed from inodeMap
    * @return the number of files that have been removed
    */
-  static long delete(
-      FSDirectory fsd, INodesInPath iip, BlocksMapUpdateInfo collectedBlocks,
-      List<INode> removedINodes, List<Long> removedUCFiles,
-      long mtime) throws IOException {
+  static long delete(FSDirectory fsd, INodesInPath iip,
+      BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes,
+      List<Long> removedUCFiles, long mtime) throws IOException {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: " + iip.getPath());
     }
-    final long filesRemoved;
+    long filesRemoved = -1;
     fsd.writeLock();
     try {
-      if (!deleteAllowed(iip, iip.getPath()) ) {
-        filesRemoved = -1;
-      } else {
+      if (deleteAllowed(iip, iip.getPath()) ) {
         List<INodeDirectory> snapshottableDirs = new ArrayList<>();
         FSDirSnapshotOp.checkSnapshot(iip.getLastINode(), snapshottableDirs);
-        filesRemoved = unprotectedDelete(fsd, iip, collectedBlocks,
-                                         removedINodes, removedUCFiles, mtime);
+        ReclaimContext context = new ReclaimContext(
+            fsd.getBlockStoragePolicySuite(), collectedBlocks, removedINodes,
+            removedUCFiles);
+        if (unprotectedDelete(fsd, iip, context, mtime)) {
+          filesRemoved = context.quotaDelta().getNsDelta();
+        }
         fsd.getFSNamesystem().removeSnapshottableDirs(snapshottableDirs);
+        fsd.updateCount(iip, context.quotaDelta(), false);
       }
     } finally {
       fsd.writeUnlock();
@@ -128,11 +131,13 @@ class FSDirDeleteOp {
     }
     List<INodeDirectory> snapshottableDirs = new ArrayList<>();
     FSDirSnapshotOp.checkSnapshot(iip.getLastINode(), snapshottableDirs);
-    long filesRemoved = unprotectedDelete(
-        fsd, iip, collectedBlocks, removedINodes, removedUCFiles, mtime);
+    boolean filesRemoved = unprotectedDelete(fsd, iip,
+        new ReclaimContext(fsd.getBlockStoragePolicySuite(),
+            collectedBlocks, removedINodes, removedUCFiles),
+        mtime);
     fsn.removeSnapshottableDirs(snapshottableDirs);
 
-    if (filesRemoved >= 0) {
+    if (filesRemoved) {
       fsn.removeLeasesAndINodes(removedUCFiles, removedINodes, false);
       fsn.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
     }
@@ -213,21 +218,18 @@ class FSDirDeleteOp {
    * Update the count at each ancestor directory with quota
    * @param fsd the FSDirectory instance
    * @param iip the inodes resolved from the path
-   * @param collectedBlocks blocks collected from the deleted path
-   * @param removedINodes inodes that should be removed from inodeMap
-   * @param removedUCFiles inodes whose leases need to be released
+   * @param reclaimContext used to collect blocks and inodes to be removed
    * @param mtime the time the inode is removed
-   * @return the number of inodes deleted; 0 if no inodes are deleted.
+   * @return true if there are inodes deleted
    */
-  private static long unprotectedDelete(
-      FSDirectory fsd, INodesInPath iip, BlocksMapUpdateInfo collectedBlocks,
-      List<INode> removedINodes, List<Long> removedUCFiles, long mtime) {
+  private static boolean unprotectedDelete(FSDirectory fsd, INodesInPath iip,
+      ReclaimContext reclaimContext, long mtime) {
     assert fsd.hasWriteLock();
 
     // check if target node exists
     INode targetNode = iip.getLastINode();
     if (targetNode == null) {
-      return -1;
+      return false;
     }
 
     // record modification
@@ -237,35 +239,24 @@ class FSDirDeleteOp {
     // Remove the node from the namespace
     long removed = fsd.removeLastINode(iip);
     if (removed == -1) {
-      return -1;
+      return false;
     }
 
     // set the parent's modification time
     final INodeDirectory parent = targetNode.getParent();
     parent.updateModificationTime(mtime, latestSnapshot);
 
-    fsd.updateCountForDelete(targetNode, iip);
-    if (removed == 0) {
-      return 0;
-    }
-
     // collect block and update quota
-    INode.ReclaimContext reclaimContext = new INode.ReclaimContext(
-        fsd.getBlockStoragePolicySuite(), collectedBlocks,
-        removedINodes, removedUCFiles);
     if (!targetNode.isInLatestSnapshot(latestSnapshot)) {
       targetNode.destroyAndCollectBlocks(reclaimContext);
     } else {
-      QuotaCounts counts = targetNode.cleanSubtree(reclaimContext,
-          CURRENT_STATE_ID, latestSnapshot);
-      removed = counts.getNameSpace();
-      fsd.updateCountNoQuotaCheck(iip, iip.length() -1, counts.negation());
+      targetNode.cleanSubtree(reclaimContext, CURRENT_STATE_ID, latestSnapshot);
     }
 
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
           + iip.getPath() + " is removed");
     }
-    return removed;
+    return true;
   }
 }

+ 10 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java

@@ -302,19 +302,18 @@ class FSDirRenameOp {
    * @param timestamp modification time
    * @param options   Rename options
    */
-  static boolean renameForEditLog(
+  static void renameForEditLog(
       FSDirectory fsd, String src, String dst, long timestamp,
       Options.Rename... options)
       throws IOException {
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
     final INodesInPath srcIIP = fsd.getINodesInPath4Write(src, false);
     final INodesInPath dstIIP = fsd.getINodesInPath4Write(dst, false);
-    boolean ret = unprotectedRenameTo(fsd, src, dst, srcIIP, dstIIP, timestamp,
+    unprotectedRenameTo(fsd, src, dst, srcIIP, dstIIP, timestamp,
         collectedBlocks, options);
     if (!collectedBlocks.getToDeleteList().isEmpty()) {
       fsd.getFSNamesystem().removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
     }
-    return ret;
   }
 
   /**
@@ -609,7 +608,7 @@ class FSDirRenameOp {
         this.srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1,
             srcChild);
         // get the counts before rename
-        withCount.getReferredINode().computeQuotaUsage(bsps, oldSrcCounts, true);
+        oldSrcCounts.add(withCount.getReferredINode().computeQuotaUsage(bsps));
       } else if (srcChildIsReference) {
         // srcChild is reference but srcChild is not in latest snapshot
         withCount = (INodeReference.WithCount) srcChild.asReference()
@@ -730,18 +729,16 @@ class FSDirRenameOp {
       Preconditions.checkState(oldDstChild != null);
       List<INode> removedINodes = new ChunkedArrayList<>();
       List<Long> removedUCFiles = new ChunkedArrayList<>();
+      INode.ReclaimContext context = new INode.ReclaimContext(bsps,
+          collectedBlocks, removedINodes, removedUCFiles);
       final boolean filesDeleted;
       if (!oldDstChild.isInLatestSnapshot(dstIIP.getLatestSnapshotId())) {
-        oldDstChild.destroyAndCollectBlocks(
-            new INode.ReclaimContext(bsps, collectedBlocks, removedINodes, removedUCFiles));
+        oldDstChild.destroyAndCollectBlocks(context);
         filesDeleted = true;
       } else {
-        filesDeleted = oldDstChild.cleanSubtree(
-            new INode.ReclaimContext(bsps, collectedBlocks, removedINodes,
-                                     removedUCFiles),
-            Snapshot.CURRENT_STATE_ID,
-            dstIIP.getLatestSnapshotId())
-            .getNameSpace() >= 0;
+        oldDstChild.cleanSubtree(context, Snapshot.CURRENT_STATE_ID,
+            dstIIP.getLatestSnapshotId());
+        filesDeleted = context.quotaDelta().getNsDelta() >= 0;
       }
       fsd.getFSNamesystem().removeLeasesAndINodes(
           removedUCFiles, removedINodes, false);
@@ -752,8 +749,7 @@ class FSDirRenameOp {
       // update the quota usage in src tree
       if (isSrcInSnapshot) {
         // get the counts after rename
-        QuotaCounts newSrcCounts = srcChild.computeQuotaUsage(bsps,
-            new QuotaCounts.Builder().build(), false);
+        QuotaCounts newSrcCounts = srcChild.computeQuotaUsage(bsps, false);
         newSrcCounts.subtract(oldSrcCounts);
         srcParent.addSpaceConsumed(newSrcCounts, false);
       }

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java

@@ -176,11 +176,13 @@ class FSDirSnapshotOp {
     }
 
     INode.BlocksMapUpdateInfo collectedBlocks = new INode.BlocksMapUpdateInfo();
-    ChunkedArrayList<INode> removedINodes = new ChunkedArrayList<INode>();
+    ChunkedArrayList<INode> removedINodes = new ChunkedArrayList<>();
+    INode.ReclaimContext context = new INode.ReclaimContext(
+        fsd.getBlockStoragePolicySuite(), collectedBlocks, removedINodes, null);
     fsd.writeLock();
     try {
-      snapshotManager.deleteSnapshot(iip, snapshotName, collectedBlocks,
-          removedINodes);
+      snapshotManager.deleteSnapshot(iip, snapshotName, context);
+      fsd.updateCount(iip, context.quotaDelta(), false);
       fsd.removeFromInodeMap(removedINodes);
     } finally {
       fsd.writeUnlock();

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

@@ -76,6 +76,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
@@ -641,6 +642,23 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  public void updateCount(INodesInPath iip, INode.QuotaDelta quotaDelta,
+      boolean check) throws QuotaExceededException {
+    QuotaCounts counts = quotaDelta.getCountsCopy();
+    updateCount(iip, iip.length() - 1, counts.negation(), check);
+    Map<INode, QuotaCounts> deltaInOtherPaths = quotaDelta.getUpdateMap();
+    for (Map.Entry<INode, QuotaCounts> entry : deltaInOtherPaths.entrySet()) {
+      INodesInPath path = INodesInPath.fromINode(entry.getKey());
+      updateCount(path, path.length() - 1, entry.getValue().negation(), check);
+    }
+    for (Map.Entry<INodeDirectory, QuotaCounts> entry :
+        quotaDelta.getQuotaDirMap().entrySet()) {
+      INodeDirectory quotaDir = entry.getKey();
+      quotaDir.getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
+          entry.getValue().negation());
+    }
+  }
+
   /**
    * Update the quota usage after deletion. The quota update is only necessary
    * when image/edits have been loaded and the file/dir to be deleted is not

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -734,9 +734,10 @@ public class FSEditLogLoader {
           renameReservedPathsOnUpgrade(deleteSnapshotOp.snapshotRoot,
               logVersion);
       INodesInPath iip = fsDir.getINodesInPath4Write(snapshotRoot);
-      fsNamesys.getSnapshotManager().deleteSnapshot(
-          iip, deleteSnapshotOp.snapshotName,
-          collectedBlocks, removedINodes);
+      fsNamesys.getSnapshotManager().deleteSnapshot(iip,
+          deleteSnapshotOp.snapshotName,
+          new INode.ReclaimContext(fsNamesys.dir.getBlockStoragePolicySuite(),
+              collectedBlocks, removedINodes, null));
       fsNamesys.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
       collectedBlocks.clear();
       fsNamesys.dir.removeFromInodeMap(removedINodes);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -886,8 +886,8 @@ public class FSImage implements Closeable {
             child.asDirectory(), counts);
       } else {
         // file or symlink: count here to reduce recursive calls.
-        child.computeQuotaUsage(bsps, childPolicyId, counts, false,
-            Snapshot.CURRENT_STATE_ID);
+        counts.add(child.computeQuotaUsage(bsps, childPolicyId, false,
+            Snapshot.CURRENT_STATE_ID));
       }
     }
       

+ 119 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -21,7 +21,10 @@ import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.util.List;
+import java.util.Map;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -222,7 +225,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
   /** Is this inode in the latest snapshot? */
   public final boolean isInLatestSnapshot(final int latestSnapshotId) {
-    if (latestSnapshotId == Snapshot.CURRENT_STATE_ID || latestSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
+    if (latestSnapshotId == Snapshot.CURRENT_STATE_ID ||
+        latestSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
       return false;
     }
     // if parent is a reference node, parent must be a renamed node. We can 
@@ -397,11 +401,10 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    *        The id of the latest snapshot before the to-be-deleted snapshot.
    *        When deleting a current inode, this parameter captures the latest
    *        snapshot.
-   * @return quota usage delta when deleting a snapshot
    */
-  public abstract QuotaCounts cleanSubtree(
-      ReclaimContext reclaimContext, final int snapshotId, int priorSnapshotId);
-  
+  public abstract void cleanSubtree(ReclaimContext reclaimContext,
+      final int snapshotId, int priorSnapshotId);
+
   /**
    * Destroy self and clear everything! If the INode is a file, this method
    * collects its blocks for further block deletion. If the INode is a
@@ -494,8 +497,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
     final byte storagePolicyId = isSymlink() ?
         HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
-    return computeQuotaUsage(bsps, storagePolicyId,
-        new QuotaCounts.Builder().build(), true, Snapshot.CURRENT_STATE_ID);
+    return computeQuotaUsage(bsps, storagePolicyId, true,
+        Snapshot.CURRENT_STATE_ID);
   }
 
   /**
@@ -521,25 +524,23 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    *
    * @param bsps Block storage policy suite to calculate intended storage type usage
    * @param blockStoragePolicyId block storage policy id of the current INode
-   * @param counts The subtree counts for returning.
    * @param useCache Whether to use cached quota usage. Note that 
    *                 {@link WithName} node never uses cache for its subtree.
    * @param lastSnapshotId {@link Snapshot#CURRENT_STATE_ID} indicates the 
    *                       computation is in the current tree. Otherwise the id
    *                       indicates the computation range for a 
    *                       {@link WithName} node.
-   * @return The same objects as the counts parameter.
+   * @return The subtree quota counts.
    */
-  public abstract QuotaCounts computeQuotaUsage(
-    BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
-    QuotaCounts counts, boolean useCache, int lastSnapshotId);
+  public abstract QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
+      byte blockStoragePolicyId, boolean useCache, int lastSnapshotId);
 
-  public final QuotaCounts computeQuotaUsage(
-    BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache) {
+  public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
+      boolean useCache) {
     final byte storagePolicyId = isSymlink() ?
         HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
-    return computeQuotaUsage(bsps, storagePolicyId, counts,
-        useCache, Snapshot.CURRENT_STATE_ID);
+    return computeQuotaUsage(bsps, storagePolicyId, useCache,
+        Snapshot.CURRENT_STATE_ID);
   }
 
   /**
@@ -803,6 +804,90 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     out.print(", " + getPermissionStatus(snapshotId));
   }
 
+  /**
+   * Information used to record quota usage delta. This data structure is
+   * usually passed along with an operation like {@link #cleanSubtree}. Note
+   * that after the operation the delta counts should be decremented from the
+   * ancestral directories' quota usage.
+   */
+  public static class QuotaDelta {
+    private final QuotaCounts counts;
+    /**
+     * The main usage of this map is to track the quota delta that should be
+     * applied to another path. This usually happens when we reclaim INodes and
+     * blocks while deleting snapshots, and hit an INodeReference. Because the
+     * quota usage for a renamed+snapshotted file/directory is counted in both
+     * the current and historical parents, any change of its quota usage may
+     * need to be propagated along its parent paths both before and after the
+     * rename.
+     */
+    private final Map<INode, QuotaCounts> updateMap;
+
+    /**
+     * When deleting a snapshot we may need to update the quota for directories
+     * with quota feature. This map is used to capture these directories and
+     * their quota usage updates.
+     */
+    private final Map<INodeDirectory, QuotaCounts> quotaDirMap;
+
+    public QuotaDelta() {
+      counts = new QuotaCounts.Builder().build();
+      updateMap = Maps.newHashMap();
+      quotaDirMap = Maps.newHashMap();
+    }
+
+    public void add(QuotaCounts update) {
+      counts.add(update);
+    }
+
+    public void addUpdatePath(INodeReference inode, QuotaCounts update) {
+      QuotaCounts c = updateMap.get(inode);
+      if (c == null) {
+        c = new QuotaCounts.Builder().build();
+        updateMap.put(inode, c);
+      }
+      c.add(update);
+    }
+
+    public void addQuotaDirUpdate(INodeDirectory dir, QuotaCounts update) {
+      Preconditions.checkState(dir.isQuotaSet());
+      QuotaCounts c = quotaDirMap.get(dir);
+      if (c == null) {
+        quotaDirMap.put(dir, update);
+      } else {
+        c.add(update);
+      }
+    }
+
+    public QuotaCounts getCountsCopy() {
+      final QuotaCounts copy = new QuotaCounts.Builder().build();
+      copy.add(counts);
+      return copy;
+    }
+
+    public void setCounts(QuotaCounts c) {
+      this.counts.setNameSpace(c.getNameSpace());
+      this.counts.setStorageSpace(c.getStorageSpace());
+      this.counts.setTypeSpaces(c.getTypeSpaces());
+    }
+
+    public long getNsDelta() {
+      long nsDelta = counts.getNameSpace();
+      for (Map.Entry<INode, QuotaCounts> entry : updateMap.entrySet()) {
+        nsDelta += entry.getValue().getNameSpace();
+      }
+      return nsDelta;
+    }
+
+    public Map<INode, QuotaCounts> getUpdateMap() {
+      return ImmutableMap.copyOf(updateMap);
+    }
+
+    public Map<INodeDirectory, QuotaCounts> getQuotaDirMap() {
+      return ImmutableMap.copyOf(quotaDirMap);
+    }
+  }
+
   /**
    * Context object to record blocks and inodes that need to be reclaimed
    */
@@ -811,6 +896,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     protected final BlocksMapUpdateInfo collectedBlocks;
     protected final List<INode> removedINodes;
     protected final List<Long> removedUCFiles;
+    /** Used to collect quota usage delta */
+    private final QuotaDelta quotaDelta;
+
     /**
      * @param bsps
      *          block storage policy suite to calculate intended storage type
@@ -830,6 +918,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
       this.collectedBlocks = collectedBlocks;
       this.removedINodes = removedINodes;
       this.removedUCFiles = removedUCFiles;
+      this.quotaDelta = new QuotaDelta();
     }
 
     public BlockStoragePolicySuite storagePolicySuite() {
@@ -839,6 +928,19 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     public BlocksMapUpdateInfo collectedBlocks() {
       return collectedBlocks;
     }
+
+    public QuotaDelta quotaDelta() {
+      return quotaDelta;
+    }
+
+    /**
+     * make a copy with the same collectedBlocks, removedINodes, and
+     * removedUCFiles but a new quotaDelta.
+     */
+    public ReclaimContext getCopy() {
+      return new ReclaimContext(bsps, collectedBlocks, removedINodes,
+          removedUCFiles);
+    }
   }
 
   /**
@@ -851,7 +953,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     private final List<Block> toDeleteList;
     
     public BlocksMapUpdateInfo() {
-      toDeleteList = new ChunkedArrayList<Block>();
+      toDeleteList = new ChunkedArrayList<>();
     }
     
     /**

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

@@ -558,7 +558,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
    */
   private void addChild(final INode node, final int insertionPoint) {
     if (children == null) {
-      children = new ArrayList<INode>(DEFAULT_FILES_PER_DIRECTORY);
+      children = new ArrayList<>(DEFAULT_FILES_PER_DIRECTORY);
     }
     node.setParent(this);
     children.add(-insertionPoint - 1, node);
@@ -570,10 +570,10 @@ public class INodeDirectory extends INodeWithAdditionalFields
 
   @Override
   public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
-      byte blockStoragePolicyId, QuotaCounts counts, boolean useCache,
-      int lastSnapshotId) {
+      byte blockStoragePolicyId, boolean useCache, int lastSnapshotId) {
     final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
 
+    QuotaCounts counts = new QuotaCounts.Builder().build();
     // we are computing the quota usage for a specific snapshot here, i.e., the
     // computation only includes files/directories that exist at the time of the
     // given snapshot
@@ -581,9 +581,10 @@ public class INodeDirectory extends INodeWithAdditionalFields
         && !(useCache && isQuotaSet())) {
       ReadOnlyList<INode> childrenList = getChildrenList(lastSnapshotId);
       for (INode child : childrenList) {
-        final byte childPolicyId = child.getStoragePolicyIDForQuota(blockStoragePolicyId);
-        child.computeQuotaUsage(bsps, childPolicyId, counts, useCache,
-            lastSnapshotId);
+        final byte childPolicyId = child.getStoragePolicyIDForQuota(
+            blockStoragePolicyId);
+        counts.add(child.computeQuotaUsage(bsps, childPolicyId, useCache,
+            lastSnapshotId));
       }
       counts.addNameSpace(1);
       return counts;
@@ -605,9 +606,10 @@ public class INodeDirectory extends INodeWithAdditionalFields
       int lastSnapshotId) {
     if (children != null) {
       for (INode child : children) {
-        final byte childPolicyId = child.getStoragePolicyIDForQuota(blockStoragePolicyId);
-        child.computeQuotaUsage(bsps, childPolicyId, counts, useCache,
-            lastSnapshotId);
+        final byte childPolicyId = child.getStoragePolicyIDForQuota(
+            blockStoragePolicyId);
+        counts.add(child.computeQuotaUsage(bsps, childPolicyId, useCache,
+            lastSnapshotId));
       }
     }
     return computeQuotaUsage4CurrentDirectory(bsps, blockStoragePolicyId,
@@ -621,7 +623,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
     // include the diff list
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     if (sf != null) {
-      sf.computeQuotaUsage4CurrentDirectory(bsps, storagePolicyId, counts);
+      counts.add(sf.computeQuotaUsage4CurrentDirectory(bsps, storagePolicyId));
     }
     return counts;
   }
@@ -710,8 +712,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
   public void undoRename4ScrParent(final INodeReference oldChild,
       final INode newChild) throws QuotaExceededException {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
-    Preconditions.checkState(sf != null,
-        "Directory does not have snapshot feature");
+    assert sf != null : "Directory does not have snapshot feature";
     sf.getDiffs().removeChild(ListType.DELETED, oldChild);
     sf.getDiffs().replaceChild(ListType.CREATED, oldChild, newChild);
     addChild(newChild, true, Snapshot.CURRENT_STATE_ID);
@@ -726,8 +727,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
       final INode deletedChild,
       int latestSnapshotId) throws QuotaExceededException {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
-    Preconditions.checkState(sf != null,
-        "Directory does not have snapshot feature");
+    assert sf != null : "Directory does not have snapshot feature";
     boolean removeDeletedChild = sf.getDiffs().removeChild(ListType.DELETED,
         deletedChild);
     int sid = removeDeletedChild ? Snapshot.CURRENT_STATE_ID : latestSnapshotId;
@@ -753,10 +753,9 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   /** Call cleanSubtree(..) recursively down the subtree. */
-  public QuotaCounts cleanSubtreeRecursively(
+  public void cleanSubtreeRecursively(
       ReclaimContext reclaimContext, final int snapshot, int prior,
       final Map<INode, INode> excludedNodes) {
-    QuotaCounts counts = new QuotaCounts.Builder().build();
     // in case of deletion snapshot, since this call happens after we modify
     // 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
@@ -765,19 +764,17 @@ public class INodeDirectory extends INodeWithAdditionalFields
     int s = snapshot != Snapshot.CURRENT_STATE_ID
         && prior != Snapshot.NO_SNAPSHOT_ID ? prior : snapshot;
     for (INode child : getChildrenList(s)) {
-      if (snapshot != Snapshot.CURRENT_STATE_ID && excludedNodes != null
-          && excludedNodes.containsKey(child)) {
-        continue;
-      } else {
-        QuotaCounts childCounts = child.cleanSubtree(reclaimContext, snapshot, prior);
-        counts.add(childCounts);
+      if (snapshot == Snapshot.CURRENT_STATE_ID || excludedNodes == null ||
+          !excludedNodes.containsKey(child)) {
+        child.cleanSubtree(reclaimContext, snapshot, prior);
       }
     }
-    return counts;
   }
 
   @Override
   public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
+    reclaimContext.quotaDelta().add(
+        new QuotaCounts.Builder().nameSpace(1).build());
     final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     if (sf != null) {
       sf.clear(reclaimContext, this);
@@ -793,30 +790,30 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
   
   @Override
-  public QuotaCounts cleanSubtree(
-      ReclaimContext reclaimContext, final int snapshotId, int priorSnapshotId) {
+  public void cleanSubtree(ReclaimContext reclaimContext, final int snapshotId,
+      int priorSnapshotId) {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     // there is snapshot data
     if (sf != null) {
-      return sf.cleanDirectory(reclaimContext, this, snapshotId,
-                               priorSnapshotId);
-    }
-    // there is no snapshot data
-    if (priorSnapshotId == Snapshot.NO_SNAPSHOT_ID
-        && snapshotId == Snapshot.CURRENT_STATE_ID) {
-      // destroy the whole subtree and collect blocks that should be deleted
-      QuotaCounts counts = new QuotaCounts.Builder().build();
-      this.computeQuotaUsage(reclaimContext.bsps, counts, true);
-      destroyAndCollectBlocks(reclaimContext);
-      return counts; 
+      sf.cleanDirectory(reclaimContext, this, snapshotId, priorSnapshotId);
     } else {
-      // process recursively down the subtree
-      QuotaCounts counts = cleanSubtreeRecursively(
-          reclaimContext, snapshotId, priorSnapshotId, null);
-      if (isQuotaSet()) {
-        getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(counts.negation());
+      // there is no snapshot data
+      if (priorSnapshotId == Snapshot.NO_SNAPSHOT_ID &&
+          snapshotId == Snapshot.CURRENT_STATE_ID) {
+        // destroy the whole subtree and collect blocks that should be deleted
+        destroyAndCollectBlocks(reclaimContext);
+      } else {
+        // make a copy the quota delta
+        QuotaCounts old = reclaimContext.quotaDelta().getCountsCopy();
+        // process recursively down the subtree
+        cleanSubtreeRecursively(reclaimContext, snapshotId, priorSnapshotId,
+            null);
+        QuotaCounts current = reclaimContext.quotaDelta().getCountsCopy();
+        current.subtract(old);
+        if (isQuotaSet()) {
+          reclaimContext.quotaDelta().addQuotaDirUpdate(this, current);
+        }
       }
-      return counts;
     }
   }
   

+ 41 - 42
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -491,37 +491,52 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override
-  public QuotaCounts cleanSubtree(
-      ReclaimContext reclaimContext, final int snapshot, int priorSnapshotId) {
+  public void cleanSubtree(ReclaimContext reclaimContext,
+      final int snapshot, int priorSnapshotId) {
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf != null) {
-      return sf.cleanFile(reclaimContext, this, snapshot, priorSnapshotId);
-    }
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-
-    if (snapshot == CURRENT_STATE_ID) {
-      if (priorSnapshotId == NO_SNAPSHOT_ID) {
-        // this only happens when deleting the current file and the file is not
-        // in any snapshot
-        computeQuotaUsage(reclaimContext.bsps, counts, false);
-        destroyAndCollectBlocks(reclaimContext);
-      } else {
-        FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
-        // when deleting the current file and the file is in snapshot, we should
-        // clean the 0-sized block if the file is UC
-        if (uc != null) {
-          uc.cleanZeroSizeBlock(this, reclaimContext.collectedBlocks);
-          if (reclaimContext.removedUCFiles != null) {
-            reclaimContext.removedUCFiles.add(getId());
+      // TODO: avoid calling getStoragePolicyID
+      sf.cleanFile(reclaimContext, this, snapshot, priorSnapshotId,
+          getStoragePolicyID());
+    } else {
+      if (snapshot == CURRENT_STATE_ID) {
+        if (priorSnapshotId == NO_SNAPSHOT_ID) {
+          // this only happens when deleting the current file and it is not
+          // in any snapshot
+          destroyAndCollectBlocks(reclaimContext);
+        } else {
+          FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
+          // when deleting the current file and it is in snapshot, we should
+          // clean the 0-sized block if the file is UC
+          if (uc != null) {
+            uc.cleanZeroSizeBlock(this, reclaimContext.collectedBlocks);
+            if (reclaimContext.removedUCFiles != null) {
+              reclaimContext.removedUCFiles.add(getId());
+            }
           }
         }
       }
     }
-    return counts;
   }
 
   @Override
   public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
+    // TODO pass in the storage policy
+    reclaimContext.quotaDelta().add(computeQuotaUsage(reclaimContext.bsps,
+        false));
+    clearFile(reclaimContext);
+    FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
+    if (sf != null) {
+      sf.getDiffs().destroyAndCollectSnapshotBlocks(
+          reclaimContext.collectedBlocks);
+      sf.clearDiffs();
+    }
+    if (isUnderConstruction() && reclaimContext.removedUCFiles != null) {
+      reclaimContext.removedUCFiles.add(getId());
+    }
+  }
+
+  public void clearFile(ReclaimContext reclaimContext) {
     if (blocks != null && reclaimContext.collectedBlocks != null) {
       for (BlockInfoContiguous blk : blocks) {
         reclaimContext.collectedBlocks.addDeleteBlock(blk);
@@ -534,15 +549,6 @@ public class INodeFile extends INodeWithAdditionalFields
     }
     clear();
     reclaimContext.removedINodes.add(this);
-    FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
-    if (sf != null) {
-      sf.getDiffs().destroyAndCollectSnapshotBlocks(
-          reclaimContext.collectedBlocks);
-      sf.clearDiffs();
-    }
-    if (isUnderConstruction() && reclaimContext.removedUCFiles != null) {
-      reclaimContext.removedUCFiles.add(getId());
-    }
   }
 
   @Override
@@ -554,18 +560,11 @@ public class INodeFile extends INodeWithAdditionalFields
   // This is the only place that needs to use the BlockStoragePolicySuite to
   // derive the intended storage type usage for quota by storage type
   @Override
-  public final QuotaCounts computeQuotaUsage(
-      BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
-      QuotaCounts counts, boolean useCache,
-      int lastSnapshotId) {
-    long nsDelta = 1;
-    counts.addNameSpace(nsDelta);
-
-    BlockStoragePolicy bsp = null;
-    if (blockStoragePolicyId != BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
-      bsp = bsps.getPolicy(blockStoragePolicyId);
-    }
+  public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
+      byte blockStoragePolicyId, boolean useCache, int lastSnapshotId) {
+    final QuotaCounts counts = new QuotaCounts.Builder().nameSpace(1).build();
 
+    final BlockStoragePolicy bsp = bsps.getPolicy(blockStoragePolicyId);
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf == null) {
       counts.add(storagespaceConsumed(bsp));
@@ -610,7 +609,7 @@ public class INodeFile extends INodeWithAdditionalFields
       final ContentSummaryComputationContext summary) {
     final ContentCounts counts = summary.getCounts();
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
-    long fileLen = 0;
+    final long fileLen;
     if (sf == null) {
       fileLen = computeFileSize();
       counts.addContent(Content.FILE, 1);

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java

@@ -37,12 +37,12 @@ public class INodeMap {
   static INodeMap newInstance(INodeDirectory rootDir) {
     // Compute the map capacity by allocating 1% of total memory
     int capacity = LightWeightGSet.computeCapacity(1, "INodeMap");
-    GSet<INode, INodeWithAdditionalFields> map
-        = new LightWeightGSet<INode, INodeWithAdditionalFields>(capacity);
+    GSet<INode, INodeWithAdditionalFields> map =
+        new LightWeightGSet<>(capacity);
     map.put(rootDir);
     return new INodeMap(map);
   }
-  
+
   /** Synchronized by external lock. */
   private final GSet<INode, INodeWithAdditionalFields> map;
   
@@ -103,7 +103,7 @@ public class INodeMap {
       @Override
       public QuotaCounts computeQuotaUsage(
           BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
-          QuotaCounts counts, boolean useCache, int lastSnapshotId) {
+          boolean useCache, int lastSnapshotId) {
         return null;
       }
 
@@ -114,9 +114,8 @@ public class INodeMap {
       }
       
       @Override
-      public QuotaCounts cleanSubtree(
+      public void cleanSubtree(
           ReclaimContext reclaimContext, int snapshotId, int priorSnapshotId) {
-          return null;
       }
 
       @Override

+ 55 - 63
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -25,13 +25,11 @@ import java.util.List;
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import com.google.common.base.Preconditions;
-import org.mortbay.log.Log;
 
 /**
  * An anonymous reference to an inode.
@@ -302,10 +300,9 @@ public abstract class INodeReference extends INode {
   }
 
   @Override // used by WithCount
-  public QuotaCounts cleanSubtree(
+  public void cleanSubtree(
       ReclaimContext reclaimContext, int snapshot, int prior) {
-    return referred.cleanSubtree(reclaimContext,
-                                 snapshot, prior);
+    referred.cleanSubtree(reclaimContext, snapshot, prior);
   }
 
   @Override // used by WithCount
@@ -322,11 +319,10 @@ public abstract class INodeReference extends INode {
   }
 
   @Override
-  public QuotaCounts computeQuotaUsage(
-    BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
-    QuotaCounts counts, boolean useCache, int lastSnapshotId) {
-    return referred.computeQuotaUsage(bsps, blockStoragePolicyId, counts,
-        useCache, lastSnapshotId);
+  public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
+      byte blockStoragePolicyId, boolean useCache, int lastSnapshotId) {
+    return referred.computeQuotaUsage(bsps, blockStoragePolicyId, useCache,
+        lastSnapshotId);
   }
 
   @Override
@@ -371,9 +367,9 @@ public abstract class INodeReference extends INode {
   
   /** An anonymous reference with reference count. */
   public static class WithCount extends INodeReference {
-    
-    private final List<WithName> withNameList = new ArrayList<WithName>();
-    
+
+    private final List<WithName> withNameList = new ArrayList<>();
+
     /**
      * Compare snapshot with IDs, where null indicates the current status thus
      * is greater than any non-null snapshot.
@@ -507,10 +503,10 @@ public abstract class INodeReference extends INode {
     @Override
     public final ContentSummaryComputationContext computeContentSummary(
         ContentSummaryComputationContext summary) {
-      //only count storagespace for WithName
-      final QuotaCounts q = new QuotaCounts.Builder().build();
-      computeQuotaUsage(summary.getBlockStoragePolicySuite(),
-          getStoragePolicyID(), q, false, lastSnapshotId);
+      // only count storagespace for WithName
+      final QuotaCounts q = computeQuotaUsage(
+          summary.getBlockStoragePolicySuite(), getStoragePolicyID(), false,
+          lastSnapshotId);
       summary.getCounts().addContent(Content.DISKSPACE, q.getStorageSpace());
       summary.getCounts().addTypeSpaces(q.getTypeSpaces());
       return summary;
@@ -518,12 +514,11 @@ public abstract class INodeReference extends INode {
 
     @Override
     public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
-        byte blockStoragePolicyId, QuotaCounts counts, boolean useCache,
-        int lastSnapshotId) {
-      // if this.lastSnapshotId < lastSnapshotId, the rename of the referred 
-      // node happened before the rename of its ancestor. This should be 
-      // impossible since for WithName node we only count its children at the 
-      // time of the rename. 
+        byte blockStoragePolicyId, boolean useCache, int lastSnapshotId) {
+      // if this.lastSnapshotId < lastSnapshotId, the rename of the referred
+      // node happened before the rename of its ancestor. This should be
+      // impossible since for WithName node we only count its children at the
+      // time of the rename.
       Preconditions.checkState(lastSnapshotId == Snapshot.CURRENT_STATE_ID
           || this.lastSnapshotId >= lastSnapshotId);
       final INode referred = this.getReferredINode().asReference()
@@ -534,13 +529,12 @@ public abstract class INodeReference extends INode {
       // been updated by changes in the current tree.
       int id = lastSnapshotId != Snapshot.CURRENT_STATE_ID ? 
           lastSnapshotId : this.lastSnapshotId;
-      return referred.computeQuotaUsage(bsps, blockStoragePolicyId, counts,
-          false, id);
+      return referred.computeQuotaUsage(bsps, blockStoragePolicyId, false, id);
     }
     
     @Override
-    public QuotaCounts cleanSubtree(
-        ReclaimContext reclaimContext, final int snapshot, int prior) {
+    public void cleanSubtree(ReclaimContext reclaimContext, final int snapshot,
+        int prior) {
       // since WithName node resides in deleted list acting as a snapshot copy,
       // the parameter snapshot must be non-null
       Preconditions.checkArgument(snapshot != Snapshot.CURRENT_STATE_ID);
@@ -552,18 +546,18 @@ public abstract class INodeReference extends INode {
       
       if (prior != Snapshot.NO_SNAPSHOT_ID
           && Snapshot.ID_INTEGER_COMPARATOR.compare(snapshot, prior) <= 0) {
-        return new QuotaCounts.Builder().build();
+        return;
       }
 
-      QuotaCounts counts = getReferredINode().cleanSubtree(reclaimContext,
-          snapshot, prior);
+      // record the old quota delta
+      QuotaCounts old = reclaimContext.quotaDelta().getCountsCopy();
+      getReferredINode().cleanSubtree(reclaimContext, snapshot, prior);
       INodeReference ref = getReferredINode().getParentReference();
       if (ref != null) {
-        try {
-          ref.addSpaceConsumed(counts.negation(), true);
-        } catch (QuotaExceededException e) {
-          Log.warn("Should not have QuotaExceededException");
-        }
+        QuotaCounts current = reclaimContext.quotaDelta().getCountsCopy();
+        current.subtract(old);
+        // we need to update the quota usage along the parent path from ref
+        reclaimContext.quotaDelta().addUpdatePath(ref, current);
       }
       
       if (snapshot < lastSnapshotId) {
@@ -571,20 +565,20 @@ public abstract class INodeReference extends INode {
         // in all the nodes existing at the time of the corresponding rename op.
         // Thus if we are deleting a snapshot before/at the snapshot associated 
         // with lastSnapshotId, we do not need to update the quota upwards.
-        counts = new QuotaCounts.Builder().build();
+        reclaimContext.quotaDelta().setCounts(old);
       }
-      return counts;
     }
     
     @Override
     public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
       int snapshot = getSelfSnapshot();
+      reclaimContext.quotaDelta().add(computeQuotaUsage(reclaimContext.bsps));
       if (removeReference(this) <= 0) {
-        getReferredINode().destroyAndCollectBlocks(reclaimContext);
+        getReferredINode().destroyAndCollectBlocks(reclaimContext.getCopy());
       } else {
         int prior = getPriorSnapshot(this);
         INode referred = getReferredINode().asReference().getReferredINode();
-        
+
         if (snapshot != Snapshot.NO_SNAPSHOT_ID) {
           if (prior != Snapshot.NO_SNAPSHOT_ID && snapshot <= prior) {
             // the snapshot to be deleted has been deleted while traversing 
@@ -598,15 +592,13 @@ public abstract class INodeReference extends INode {
             // 5. delete snapshot s2
             return;
           }
-          try {
-            QuotaCounts counts = referred.cleanSubtree(reclaimContext,
-                snapshot, prior);
-            INodeReference ref = getReferredINode().getParentReference();
-            if (ref != null) {
-              ref.addSpaceConsumed(counts.negation(), true);
-            }
-          } catch (QuotaExceededException e) {
-            LOG.error("should not exceed quota while snapshot deletion", e);
+          ReclaimContext newCtx = reclaimContext.getCopy();
+          referred.cleanSubtree(newCtx, snapshot, prior);
+          INodeReference ref = getReferredINode().getParentReference();
+          if (ref != null) {
+            // we need to update the quota usage along the parent path from ref
+            reclaimContext.quotaDelta().addUpdatePath(ref,
+                newCtx.quotaDelta().getCountsCopy());
           }
         }
       }
@@ -654,14 +646,11 @@ public abstract class INodeReference extends INode {
     }
     
     @Override
-    public QuotaCounts cleanSubtree(
-        ReclaimContext reclaimContext, int snapshot, int prior) {
+    public void cleanSubtree(ReclaimContext reclaimContext, int snapshot,
+        int prior) {
       if (snapshot == Snapshot.CURRENT_STATE_ID
           && prior == Snapshot.NO_SNAPSHOT_ID) {
-        QuotaCounts counts = new QuotaCounts.Builder().build();
-        this.computeQuotaUsage(reclaimContext.bsps, counts, true);
         destroyAndCollectBlocks(reclaimContext);
-        return counts;
       } else {
         // if prior is NO_SNAPSHOT_ID, we need to check snapshot belonging to 
         // the previous WithName instance
@@ -674,9 +663,9 @@ public abstract class INodeReference extends INode {
         if (snapshot != Snapshot.CURRENT_STATE_ID
             && prior != Snapshot.NO_SNAPSHOT_ID
             && Snapshot.ID_INTEGER_COMPARATOR.compare(snapshot, prior) <= 0) {
-          return new QuotaCounts.Builder().build();
+          return;
         }
-        return getReferredINode().cleanSubtree(reclaimContext, snapshot, prior);
+        getReferredINode().cleanSubtree(reclaimContext, snapshot, prior);
       }
     }
     
@@ -693,8 +682,15 @@ public abstract class INodeReference extends INode {
      */
     @Override
     public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
+      // since we count everything of the subtree for the quota usage of a
+      // dst reference node, here we should just simply do a quota computation.
+      // then to avoid double counting, we pass a different QuotaDelta to other
+      // calls
+      reclaimContext.quotaDelta().add(computeQuotaUsage(reclaimContext.bsps));
+      ReclaimContext newCtx = reclaimContext.getCopy();
+
       if (removeReference(this) <= 0) {
-        getReferredINode().destroyAndCollectBlocks(reclaimContext);
+        getReferredINode().destroyAndCollectBlocks(newCtx);
       } else {
         // we will clean everything, including files, directories, and 
         // snapshots, that were created after this prior snapshot
@@ -716,18 +712,14 @@ public abstract class INodeReference extends INode {
           // when calling cleanSubtree of the referred node, since we
           // compute quota usage updates before calling this destroy
           // function, we use true for countDiffChange
-          referred.cleanSubtree(reclaimContext, snapshot, prior);
+          referred.cleanSubtree(newCtx, snapshot, prior);
         } else if (referred.isDirectory()) {
           // similarly, if referred is a directory, it must be an
           // INodeDirectory with snapshot
           INodeDirectory dir = referred.asDirectory();
           Preconditions.checkState(dir.isWithSnapshot());
-          try {
-            DirectoryWithSnapshotFeature.destroyDstSubtree(
-                reclaimContext, dir, snapshot, prior);
-          } catch (QuotaExceededException e) {
-            LOG.error("should not exceed quota while snapshot deletion", e);
-          }
+          DirectoryWithSnapshotFeature.destroyDstSubtree(newCtx, dir,
+              snapshot, prior);
         }
       }
     }

+ 7 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java

@@ -72,26 +72,25 @@ public class INodeSymlink extends INodeWithAdditionalFields {
   }
   
   @Override
-  public QuotaCounts cleanSubtree(
-      ReclaimContext reclaimContext, final int snapshotId, int priorSnapshotId) {
+  public void cleanSubtree(ReclaimContext reclaimContext, final int snapshotId,
+      int priorSnapshotId) {
     if (snapshotId == Snapshot.CURRENT_STATE_ID
         && priorSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
       destroyAndCollectBlocks(reclaimContext);
     }
-    return new QuotaCounts.Builder().nameSpace(1).build();
   }
   
   @Override
   public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
     reclaimContext.removedINodes.add(this);
+    reclaimContext.quotaDelta().add(
+        new QuotaCounts.Builder().nameSpace(1).build());
   }
 
   @Override
-  public QuotaCounts computeQuotaUsage(
-      BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
-      QuotaCounts counts, boolean useCache, int lastSnapshotId) {
-    counts.addNameSpace(1);
-    return counts;
+  public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
+      byte blockStoragePolicyId, boolean useCache, int lastSnapshotId) {
+    return new QuotaCounts.Builder().nameSpace(1).build();
   }
 
   @Override
@@ -108,15 +107,6 @@ public class INodeSymlink extends INodeWithAdditionalFields {
     out.println();
   }
 
-  /**
-   * getAclFeature is not overridden because it is needed for resolving
-   * symlinks.
-  @Override
-  final AclFeature getAclFeature(int snapshotId) {
-    throw new UnsupportedOperationException("ACLs are not supported on symlinks");
-  }
-  */
-
   @Override
   public void removeAclFeature() {
     throw new UnsupportedOperationException("ACLs are not supported on symlinks");

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/QuotaCounts.java

@@ -78,14 +78,16 @@ public class QuotaCounts {
     this.tsCounts = builder.tsCounts;
   }
 
-  public void add(QuotaCounts that) {
+  public QuotaCounts add(QuotaCounts that) {
     this.nsSsCounts.add(that.nsSsCounts);
     this.tsCounts.add(that.tsCounts);
+    return this;
   }
 
-  public void subtract(QuotaCounts that) {
+  public QuotaCounts subtract(QuotaCounts that) {
     this.nsSsCounts.subtract(that.nsSsCounts);
     this.tsCounts.subtract(that.tsCounts);
+    return this;
   }
 
   /**

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

@@ -19,13 +19,9 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.List;
 
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
-import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 
 import com.google.common.base.Preconditions;
@@ -115,7 +111,7 @@ abstract class AbstractINodeDiff<N extends INode,
   }
 
   /** Combine the posterior diff and collect blocks for deletion. */
-  abstract QuotaCounts combinePosteriorAndCollectBlocks(
+  abstract void combinePosteriorAndCollectBlocks(
       INode.ReclaimContext reclaimContext, final N currentINode,
       final D posterior);
   
@@ -123,11 +119,9 @@ abstract class AbstractINodeDiff<N extends INode,
    * Delete and clear self.
    * @param reclaimContext blocks and inodes that need to be reclaimed
    * @param currentINode The inode where the deletion happens.
-   *
-   * @return usage delta
    */
-  abstract QuotaCounts destroyDiffAndCollectBlocks(
-      INode.ReclaimContext reclaimContext, final N currentINode);
+  abstract void destroyDiffAndCollectBlocks(INode.ReclaimContext reclaimContext,
+      final N currentINode);
 
   @Override
   public String toString() {

+ 9 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java

@@ -24,7 +24,6 @@ import java.util.List;
 
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
-import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 
 /**
  * A list of snapshot diffs for storing snapshot data.
@@ -63,23 +62,20 @@ abstract class AbstractINodeDiffList<N extends INode,
    * @param reclaimContext blocks and inodes that need to be reclaimed
    * @param snapshot The id of the snapshot to be deleted
    * @param prior The id of the snapshot taken before the to-be-deleted snapshot
-   * @return delta in namespace.
+   * @param currentINode the inode where the snapshot diff is deleted
    */
-  public final QuotaCounts deleteSnapshotDiff(
-      INode.ReclaimContext reclaimContext, final int snapshot, final int prior,
-      final N currentINode) {
+  public final void deleteSnapshotDiff(INode.ReclaimContext reclaimContext,
+      final int snapshot, final int prior, final N currentINode) {
     int snapshotIndex = Collections.binarySearch(diffs, snapshot);
-    
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-    D removed = null;
+
+    D removed;
     if (snapshotIndex == 0) {
       if (prior != Snapshot.NO_SNAPSHOT_ID) { // there is still snapshot before
         // set the snapshot to latestBefore
         diffs.get(snapshotIndex).setSnapshotId(prior);
       } else { // there is no snapshot before
         removed = diffs.remove(0);
-        counts.add(removed.destroyDiffAndCollectBlocks(reclaimContext,
-            currentINode));
+        removed.destroyDiffAndCollectBlocks(reclaimContext, currentINode);
       }
     } else if (snapshotIndex > 0) {
       final AbstractINodeDiff<N, A, D> previous = diffs.get(snapshotIndex - 1);
@@ -92,13 +88,12 @@ abstract class AbstractINodeDiffList<N extends INode,
           previous.snapshotINode = removed.snapshotINode;
         }
 
-        counts.add(previous.combinePosteriorAndCollectBlocks(reclaimContext,
-            currentINode, removed));
+        previous.combinePosteriorAndCollectBlocks(reclaimContext, currentINode,
+            removed);
         previous.setPosterior(removed.getPosterior());
         removed.setPosterior(null);
       }
     }
-    return counts;
   }
 
   /** Add an {@link AbstractINodeDiff} for the given snapshot. */
@@ -107,7 +102,7 @@ abstract class AbstractINodeDiffList<N extends INode,
   }
 
   /** Append the diff at the end of the list. */
-  private final D addLast(D diff) {
+  private D addLast(D diff) {
     final D last = getLast();
     diffs.add(diff);
     if (last != null) {

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

@@ -33,14 +33,12 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.Content;
 import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
 import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.SnapshotAndINode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 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.WithName;
-import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.Time;
@@ -214,18 +212,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
     } else {
       final Snapshot snapshot = snapshotsByNames.get(i);
       int prior = Snapshot.findLatestSnapshot(snapshotRoot, snapshot.getId());
-      try {
-        QuotaCounts counts = snapshotRoot.cleanSubtree(reclaimContext,
-            snapshot.getId(), prior);
-        INodeDirectory parent = snapshotRoot.getParent();
-        if (parent != null) {
-          // there will not be any WithName node corresponding to the deleted
-          // snapshot, thus only update the quota usage in the current tree
-          parent.addSpaceConsumed(counts.negation(), true);
-        }
-      } catch(QuotaExceededException e) {
-        INode.LOG.error("BUG: removeSnapshot increases namespace usage.", e);
-      }
+      snapshotRoot.cleanSubtree(reclaimContext, snapshot.getId(), prior);
       // remove from snapshotsByNames after successfully cleaning the subtree
       snapshotsByNames.remove(i);
       return snapshot;

+ 48 - 69
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java

@@ -49,6 +49,8 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.base.Preconditions;
 
+import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
+
 /**
  * Feature used to store and process the snapshot diff information for a
  * directory. In particular, it contains a directory diff list recording changes
@@ -95,30 +97,24 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     /** clear the created list */
-    private QuotaCounts destroyCreatedList(
-        INode.ReclaimContext reclaimContext, final INodeDirectory currentINode) {
-      QuotaCounts counts = new QuotaCounts.Builder().build();
+    private void destroyCreatedList(INode.ReclaimContext reclaimContext,
+        final INodeDirectory currentINode) {
       final List<INode> createdList = getList(ListType.CREATED);
       for (INode c : createdList) {
-        c.computeQuotaUsage(reclaimContext.storagePolicySuite(), counts, true);
         c.destroyAndCollectBlocks(reclaimContext);
         // c should be contained in the children list, remove it
         currentINode.removeChild(c);
       }
       createdList.clear();
-      return counts;
     }
 
     /** clear the deleted list */
-    private QuotaCounts destroyDeletedList(INode.ReclaimContext reclaimContext) {
-      QuotaCounts counts = new QuotaCounts.Builder().build();
+    private void destroyDeletedList(INode.ReclaimContext reclaimContext) {
       final List<INode> deletedList = getList(ListType.DELETED);
       for (INode d : deletedList) {
-        d.computeQuotaUsage(reclaimContext.storagePolicySuite(), counts, false);
         d.destroyAndCollectBlocks(reclaimContext);
       }
       deletedList.clear();
-      return counts;
     }
 
     /** Serialize {@link #created} */
@@ -202,22 +198,19 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     @Override
-    QuotaCounts combinePosteriorAndCollectBlocks(
+    void combinePosteriorAndCollectBlocks(
         final INode.ReclaimContext reclaimContext,
         final INodeDirectory currentDir,
         final DirectoryDiff posterior) {
-      final QuotaCounts counts = new QuotaCounts.Builder().build();
       diff.combinePosterior(posterior.diff, new Diff.Processor<INode>() {
         /** Collect blocks for deleted files. */
         @Override
         public void process(INode inode) {
           if (inode != null) {
-            inode.computeQuotaUsage(reclaimContext.storagePolicySuite(), counts, false);
             inode.destroyAndCollectBlocks(reclaimContext);
           }
         }
       });
-      return counts;
     }
 
     /**
@@ -311,16 +304,14 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     @Override
-    QuotaCounts destroyDiffAndCollectBlocks(
+    void destroyDiffAndCollectBlocks(
         INode.ReclaimContext reclaimContext, INodeDirectory currentINode) {
       // this diff has been deleted
-      QuotaCounts counts = new QuotaCounts.Builder().build();
-      counts.add(diff.destroyDeletedList(reclaimContext));
+      diff.destroyDeletedList(reclaimContext);
       INodeDirectoryAttributes snapshotINode = getSnapshotINode();
       if (snapshotINode != null && snapshotINode.getAclFeature() != null) {
         AclStorage.removeAclFeature(snapshotINode.getAclFeature());
       }
-      return counts;
     }
   }
 
@@ -381,7 +372,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           return diffList.get(i).getSnapshotId();
         }
       }
-      return Snapshot.NO_SNAPSHOT_ID;
+      return NO_SNAPSHOT_ID;
     }
   }
   
@@ -389,7 +380,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     if (diffList == null || diffList.size() == 0) {
       return null;
     }
-    Map<INode, INode> map = new HashMap<INode, INode>(diffList.size());
+    Map<INode, INode> map = new HashMap<>(diffList.size());
     for (INode node : diffList) {
       map.put(node, node);
     }
@@ -399,22 +390,19 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
   /**
    * Destroy a subtree under a DstReference node.
    */
-  public static void destroyDstSubtree(
-      INode.ReclaimContext reclaimContext, INode inode, final int snapshot,
-      final int prior) throws QuotaExceededException {
-    Preconditions.checkArgument(prior != Snapshot.NO_SNAPSHOT_ID);
+  public static void destroyDstSubtree(INode.ReclaimContext reclaimContext,
+      INode inode, final int snapshot, final int prior) {
+    Preconditions.checkArgument(prior != NO_SNAPSHOT_ID);
     if (inode.isReference()) {
       if (inode instanceof INodeReference.WithName
           && snapshot != Snapshot.CURRENT_STATE_ID) {
         // this inode has been renamed before the deletion of the DstReference
         // subtree
-        inode.cleanSubtree(reclaimContext,
-            snapshot, prior);
-      } else { 
+        inode.cleanSubtree(reclaimContext, snapshot, prior);
+      } else {
         // for DstReference node, continue this process to its subtree
         destroyDstSubtree(reclaimContext,
-                          inode.asReference().getReferredINode(), snapshot,
-                          prior);
+            inode.asReference().getReferredINode(), snapshot, prior);
       }
     } else if (inode.isFile()) {
       inode.cleanSubtree(reclaimContext, snapshot, prior);
@@ -455,13 +443,10 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
    * @param inode The inode to clean.
    * @param post The post snapshot.
    * @param prior The id of the prior snapshot.
-   * @return Quota usage update.
    */
-  private static QuotaCounts cleanDeletedINode(
-      INode.ReclaimContext reclaimContext, INode inode, final int post,
-      final int prior) {
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-    Deque<INode> queue = new ArrayDeque<INode>();
+  private static void cleanDeletedINode(INode.ReclaimContext reclaimContext,
+      INode inode, final int post, final int prior) {
+    Deque<INode> queue = new ArrayDeque<>();
     queue.addLast(inode);
     while (!queue.isEmpty()) {
       INode topNode = queue.pollFirst();
@@ -474,7 +459,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         // prior, we should treat it as regular file/dir
       } else if (topNode.isFile() && topNode.asFile().isWithSnapshot()) {
         INodeFile file = topNode.asFile();
-        counts.add(file.getDiffs().deleteSnapshotDiff(reclaimContext, post, prior, file));
+        file.getDiffs().deleteSnapshotDiff(reclaimContext, post, prior, file);
       } else if (topNode.isDirectory()) {
         INodeDirectory dir = topNode.asDirectory();
         ChildrenDiff priorChildrenDiff = null;
@@ -485,22 +470,19 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           DirectoryDiff priorDiff = sf.getDiffs().getDiffById(prior);
           if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
             priorChildrenDiff = priorDiff.getChildrenDiff();
-            counts.add(priorChildrenDiff.destroyCreatedList(reclaimContext,
-                dir));
+            priorChildrenDiff.destroyCreatedList(reclaimContext, dir);
           }
         }
-        
+
         for (INode child : dir.getChildrenList(prior)) {
-          if (priorChildrenDiff != null
-              && priorChildrenDiff.search(ListType.DELETED,
-                  child.getLocalNameBytes()) != null) {
+          if (priorChildrenDiff != null && priorChildrenDiff.search(
+              ListType.DELETED, child.getLocalNameBytes()) != null) {
             continue;
           }
           queue.addLast(child);
         }
       }
     }
-    return counts;
   }
 
   /** Diff list sorted by snapshot IDs, i.e. in chronological order. */
@@ -626,13 +608,14 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
   }
 
   public QuotaCounts computeQuotaUsage4CurrentDirectory(
-      BlockStoragePolicySuite bsps, byte storagePolicyId,
-      QuotaCounts counts) {
+      BlockStoragePolicySuite bsps, byte storagePolicyId) {
+    final QuotaCounts counts = new QuotaCounts.Builder().build();
     for(DirectoryDiff d : diffs) {
       for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
-        final byte childPolicyId = deleted.getStoragePolicyIDForQuota(storagePolicyId);
-        deleted.computeQuotaUsage(bsps, childPolicyId, counts, false,
-            Snapshot.CURRENT_STATE_ID);
+        final byte childPolicyId = deleted.getStoragePolicyIDForQuota(
+            storagePolicyId);
+        counts.add(deleted.computeQuotaUsage(bsps, childPolicyId, false,
+            Snapshot.CURRENT_STATE_ID));
       }
     }
     return counts;
@@ -706,28 +689,26 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
   }
 
-  public QuotaCounts cleanDirectory(
-      INode.ReclaimContext reclaimContext, final INodeDirectory currentINode,
-      final int snapshot, int prior) {
-    QuotaCounts counts = new QuotaCounts.Builder().build();
+  public void cleanDirectory(INode.ReclaimContext reclaimContext,
+      final INodeDirectory currentINode, final int snapshot, int prior) {
     Map<INode, INode> priorCreated = null;
     Map<INode, INode> priorDeleted = null;
+    QuotaCounts old = reclaimContext.quotaDelta().getCountsCopy();
     if (snapshot == Snapshot.CURRENT_STATE_ID) { // delete the current directory
       currentINode.recordModification(prior);
       // delete everything in created list
       DirectoryDiff lastDiff = diffs.getLast();
       if (lastDiff != null) {
-        counts.add(lastDiff.diff.destroyCreatedList(reclaimContext,
-            currentINode));
+        lastDiff.diff.destroyCreatedList(reclaimContext, currentINode);
       }
-      counts.add(currentINode.cleanSubtreeRecursively(reclaimContext,
-          snapshot, prior, priorDeleted));
+      currentINode.cleanSubtreeRecursively(reclaimContext, snapshot, prior,
+          null);
     } else {
       // update prior
       prior = getDiffs().updatePrior(snapshot, prior);
       // if there is a snapshot diff associated with prior, we need to record
       // its original created and deleted list before deleting post
-      if (prior != Snapshot.NO_SNAPSHOT_ID) {
+      if (prior != NO_SNAPSHOT_ID) {
         DirectoryDiff priorDiff = this.getDiffs().getDiffById(prior);
         if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
           List<INode> cList = priorDiff.diff.getList(ListType.CREATED);
@@ -736,14 +717,14 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           priorDeleted = cloneDiffList(dList);
         }
       }
-      
-      counts.add(getDiffs().deleteSnapshotDiff(reclaimContext, snapshot, prior,
-          currentINode));
-      counts.add(currentINode.cleanSubtreeRecursively(reclaimContext,
-          snapshot, prior, priorDeleted));
+
+      getDiffs().deleteSnapshotDiff(reclaimContext, snapshot, prior,
+          currentINode);
+      currentINode.cleanSubtreeRecursively(reclaimContext, snapshot, prior,
+          priorDeleted);
 
       // check priorDiff again since it may be created during the diff deletion
-      if (prior != Snapshot.NO_SNAPSHOT_ID) {
+      if (prior != NO_SNAPSHOT_ID) {
         DirectoryDiff priorDiff = this.getDiffs().getDiffById(prior);
         if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
           // For files/directories created between "prior" and "snapshot", 
@@ -756,8 +737,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
             for (INode cNode : priorDiff.getChildrenDiff().getList(
                 ListType.CREATED)) {
               if (priorCreated.containsKey(cNode)) {
-                counts.add(cNode.cleanSubtree(reclaimContext,
-                    snapshot, Snapshot.NO_SNAPSHOT_ID));
+                cNode.cleanSubtree(reclaimContext, snapshot, NO_SNAPSHOT_ID);
               }
             }
           }
@@ -773,18 +753,17 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           for (INode dNode : priorDiff.getChildrenDiff().getList(
               ListType.DELETED)) {
             if (priorDeleted == null || !priorDeleted.containsKey(dNode)) {
-              counts.add(cleanDeletedINode(reclaimContext,
-                  dNode, snapshot, prior));
+              cleanDeletedINode(reclaimContext, dNode, snapshot, prior);
             }
           }
         }
       }
     }
 
+    QuotaCounts current = reclaimContext.quotaDelta().getCountsCopy();
+    current.subtract(old);
     if (currentINode.isQuotaSet()) {
-      currentINode.getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
-          counts.negation());
+      reclaimContext.quotaDelta().addQuotaDirUpdate(currentINode, current);
     }
-    return counts;
   }
 }

+ 10 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java

@@ -27,7 +27,6 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
-import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 
 /**
@@ -79,13 +78,12 @@ public class FileDiff extends
   }
 
   @Override
-  QuotaCounts combinePosteriorAndCollectBlocks(
+  void combinePosteriorAndCollectBlocks(
       INode.ReclaimContext reclaimContext, INodeFile currentINode,
       FileDiff posterior) {
     FileWithSnapshotFeature sf = currentINode.getFileWithSnapshotFeature();
     assert sf != null : "FileWithSnapshotFeature is null";
-    return sf.updateQuotaAndCollectBlocks(reclaimContext,
-        currentINode, posterior);
+    sf.updateQuotaAndCollectBlocks(reclaimContext, currentINode, posterior);
   }
   
   @Override
@@ -109,18 +107,20 @@ public class FileDiff extends
   }
 
   @Override
-  QuotaCounts destroyDiffAndCollectBlocks(
-      INode.ReclaimContext reclaimContext, INodeFile currentINode) {
-    return currentINode.getFileWithSnapshotFeature()
-        .updateQuotaAndCollectBlocks(reclaimContext, currentINode, this);
+  void destroyDiffAndCollectBlocks(INode.ReclaimContext reclaimContext,
+      INodeFile currentINode) {
+    currentINode.getFileWithSnapshotFeature().updateQuotaAndCollectBlocks(
+        reclaimContext, currentINode, this);
   }
 
   public void destroyAndCollectSnapshotBlocks(
       BlocksMapUpdateInfo collectedBlocks) {
-    if(blocks == null || collectedBlocks == null)
+    if (blocks == null || collectedBlocks == null) {
       return;
-    for(BlockInfoContiguous blk : blocks)
+    }
+    for (BlockInfoContiguous blk : blocks) {
       collectedBlocks.addDeleteBlock(blk);
+    }
     blocks = null;
   }
 }

+ 13 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java

@@ -45,21 +45,23 @@ public class FileDiffList extends
 
   public void destroyAndCollectSnapshotBlocks(
       BlocksMapUpdateInfo collectedBlocks) {
-    for(FileDiff d : asList())
+    for (FileDiff d : asList()) {
       d.destroyAndCollectSnapshotBlocks(collectedBlocks);
+    }
   }
 
   public void saveSelf2Snapshot(int latestSnapshotId, INodeFile iNodeFile,
       INodeFileAttributes snapshotCopy, boolean withBlocks) {
     final FileDiff diff =
         super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy);
-    if(withBlocks)  // Store blocks if this is the first update
+    if (withBlocks) {  // Store blocks if this is the first update
       diff.setBlocks(iNodeFile.getBlocks());
+    }
   }
 
   public BlockInfoContiguous[] findEarlierSnapshotBlocks(int snapshotId) {
     assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
-    if(snapshotId == Snapshot.CURRENT_STATE_ID) {
+    if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       return null;
     }
     List<FileDiff> diffs = this.asList();
@@ -76,15 +78,15 @@ public class FileDiffList extends
 
   public BlockInfoContiguous[] findLaterSnapshotBlocks(int snapshotId) {
     assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
-    if(snapshotId == Snapshot.CURRENT_STATE_ID) {
+    if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       return null;
     }
     List<FileDiff> diffs = this.asList();
     int i = Collections.binarySearch(diffs, snapshotId);
     BlockInfoContiguous[] blocks = null;
-    for(i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
+    for (i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
       blocks = diffs.get(i).getBlocks();
-      if(blocks != null) {
+      if (blocks != null) {
         break;
       }
     }
@@ -99,7 +101,7 @@ public class FileDiffList extends
   void combineAndCollectSnapshotBlocks(
       INode.ReclaimContext reclaimContext, INodeFile file, FileDiff removed) {
     BlockInfoContiguous[] removedBlocks = removed.getBlocks();
-    if(removedBlocks == null) {
+    if (removedBlocks == null) {
       FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
       assert sf != null : "FileWithSnapshotFeature is null";
       if(sf.isCurrentFileDeleted())
@@ -109,8 +111,9 @@ public class FileDiffList extends
     int p = getPrior(removed.getSnapshotId(), true);
     FileDiff earlierDiff = p == Snapshot.NO_SNAPSHOT_ID ? null : getDiffById(p);
     // Copy blocks to the previous snapshot if not set already
-    if(earlierDiff != null)
+    if (earlierDiff != null) {
       earlierDiff.setBlocks(removedBlocks);
+    }
     BlockInfoContiguous[] earlierBlocks =
         (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks());
     // Find later snapshot (or file itself) with blocks
@@ -128,13 +131,13 @@ public class FileDiffList extends
     // Check if last block is part of truncate recovery
     BlockInfoContiguous lastBlock = file.getLastBlock();
     Block dontRemoveBlock = null;
-    if(lastBlock != null && lastBlock.getBlockUCState().equals(
+    if (lastBlock != null && lastBlock.getBlockUCState().equals(
         HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
       dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock)
           .getTruncateBlock();
     }
     // Collect the remaining blocks of the file, ignoring truncate block
-    for(;i < removedBlocks.length; i++) {
+    for (;i < removedBlocks.length; i++) {
       if(dontRemoveBlock == null || !removedBlocks[i].equals(dontRemoveBlock)) {
         reclaimContext.collectedBlocks().addDeleteBlock(removedBlocks[i]);
       }

+ 15 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java

@@ -116,21 +116,25 @@ public class FileWithSnapshotFeature implements INode.Feature {
     return (isCurrentFileDeleted()? "(DELETED), ": ", ") + diffs;
   }
   
-  public QuotaCounts cleanFile(INode.ReclaimContext reclaimContext,
-      final INodeFile file, final int snapshotId,
-      int priorSnapshotId) {
+  public void cleanFile(INode.ReclaimContext reclaimContext,
+      final INodeFile file, final int snapshotId, int priorSnapshotId,
+      byte storagePolicyId) {
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       // delete the current file while the file has snapshot feature
       if (!isCurrentFileDeleted()) {
         file.recordModification(priorSnapshotId);
         deleteCurrentFile();
       }
+      final BlockStoragePolicy policy = reclaimContext.storagePolicySuite()
+          .getPolicy(storagePolicyId);
+      QuotaCounts old = file.storagespaceConsumed(policy);
       collectBlocksAndClear(reclaimContext, file);
-      return new QuotaCounts.Builder().build();
+      QuotaCounts current = file.storagespaceConsumed(policy);
+      reclaimContext.quotaDelta().add(old.subtract(current));
     } else { // delete the snapshot
       priorSnapshotId = getDiffs().updatePrior(snapshotId, priorSnapshotId);
-      return diffs.deleteSnapshotDiff(reclaimContext,
-          snapshotId, priorSnapshotId, file);
+      diffs.deleteSnapshotDiff(reclaimContext, snapshotId, priorSnapshotId,
+          file);
     }
   }
   
@@ -138,8 +142,8 @@ public class FileWithSnapshotFeature implements INode.Feature {
     this.diffs.clear();
   }
   
-  public QuotaCounts updateQuotaAndCollectBlocks(
-      INode.ReclaimContext reclaimContext, INodeFile file, FileDiff removed) {
+  public void updateQuotaAndCollectBlocks(INode.ReclaimContext reclaimContext,
+      INodeFile file, FileDiff removed) {
     byte storagePolicyID = file.getStoragePolicyID();
     BlockStoragePolicy bsp = null;
     if (storagePolicyID != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
@@ -179,8 +183,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
     getDiffs().combineAndCollectSnapshotBlocks(reclaimContext, file, removed);
 
     QuotaCounts current = file.storagespaceConsumed(bsp);
-    oldCounts.subtract(current);
-    return oldCounts;
+    reclaimContext.quotaDelta().add(oldCounts.subtract(current));
   }
 
   /**
@@ -191,7 +194,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
       INode.ReclaimContext reclaimContext, final INodeFile file) {
     // check if everything is deleted.
     if (isCurrentFileDeleted() && getDiffs().asList().isEmpty()) {
-      file.destroyAndCollectBlocks(reclaimContext);
+      file.clearFile(reclaimContext);
       return;
     }
     // find max file size.
@@ -199,7 +202,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
     FileDiff diff = getDiffs().getLast();
     if (isCurrentFileDeleted()) {
       max = diff == null? 0: diff.getFileSize();
-    } else { 
+    } else {
       max = file.computeFileSize();
     }
 

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

@@ -40,7 +40,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
 import org.apache.hadoop.metrics2.util.MBeans;
@@ -211,7 +210,6 @@ public class SnapshotManager implements SnapshotStatsMXBean {
       // We have reached the maximum allowable snapshot ID and since we don't
       // handle rollover we will fail all subsequent snapshot creation
       // requests.
-      //
       throw new SnapshotException(
           "Failed to create the snapshot. The FileSystem has run out of " +
           "snapshot IDs and ID rollover is not supported.");
@@ -228,17 +226,13 @@ public class SnapshotManager implements SnapshotStatsMXBean {
   /**
    * Delete a snapshot for a snapshottable directory
    * @param snapshotName Name of the snapshot to be deleted
-   * @param collectedBlocks Used to collect information to update blocksMap
-   * @throws IOException
+   * @param reclaimContext Used to collect information to reclaim blocks
+   *                       and inodes
    */
   public void deleteSnapshot(final INodesInPath iip, final String snapshotName,
-      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
-      throws IOException {
+      INode.ReclaimContext reclaimContext) throws IOException {
     INodeDirectory srcRoot = getSnapshottableRoot(iip);
-    srcRoot.removeSnapshot(
-        new INode.ReclaimContext(fsdir.getBlockStoragePolicySuite(),
-                                 collectedBlocks, removedINodes, null),
-        snapshotName);
+    srcRoot.removeSnapshot(reclaimContext, snapshotName);
     numSnapshots.getAndDecrement();
   }
 

+ 10 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaByStorageType.java

@@ -47,7 +47,6 @@ public class TestQuotaByStorageType {
   private static final long seed = 0L;
   private static final Path dir = new Path("/TestQuotaByStorageType");
 
-  private Configuration conf;
   private MiniDFSCluster cluster;
   private FSDirectory fsdir;
   private DistributedFileSystem dfs;
@@ -57,7 +56,7 @@ public class TestQuotaByStorageType {
 
   @Before
   public void setUp() throws Exception {
-    conf = new Configuration();
+    Configuration conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
 
     // Setup a 3-node cluster and configure
@@ -194,8 +193,8 @@ public class TestQuotaByStorageType {
         .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
     assertEquals(0, storageTypeConsumed);
 
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-    fnode.computeQuotaUsage(fsn.getBlockManager().getStoragePolicySuite(), counts, true);
+    QuotaCounts counts = fnode.computeQuotaUsage(
+        fsn.getBlockManager().getStoragePolicySuite(), true);
     assertEquals(fnode.dumpTreeRecursively().toString(), 0,
         counts.getTypeSpaces().get(StorageType.SSD));
 
@@ -453,8 +452,8 @@ public class TestQuotaByStorageType {
     assertEquals(0, cntAfterDelete.getStorageSpace());
 
     // Validate the computeQuotaUsage()
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-    fnode.computeQuotaUsage(fsn.getBlockManager().getStoragePolicySuite(), counts, true);
+    QuotaCounts counts = fnode.computeQuotaUsage(
+        fsn.getBlockManager().getStoragePolicySuite(), true);
     assertEquals(fnode.dumpTreeRecursively().toString(), 1,
         counts.getNameSpace());
     assertEquals(fnode.dumpTreeRecursively().toString(), 0,
@@ -565,8 +564,8 @@ public class TestQuotaByStorageType {
         .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
     assertEquals(file1Len, ssdConsumed);
 
-    QuotaCounts counts1 = new QuotaCounts.Builder().build();
-    sub1Node.computeQuotaUsage(fsn.getBlockManager().getStoragePolicySuite(), counts1, true);
+    QuotaCounts counts1 = sub1Node.computeQuotaUsage(
+        fsn.getBlockManager().getStoragePolicySuite(), true);
     assertEquals(sub1Node.dumpTreeRecursively().toString(), file1Len,
         counts1.getTypeSpaces().get(StorageType.SSD));
 
@@ -583,8 +582,8 @@ public class TestQuotaByStorageType {
         .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
     assertEquals(0, ssdConsumed);
 
-    QuotaCounts counts2 = new QuotaCounts.Builder().build();
-    sub1Node.computeQuotaUsage(fsn.getBlockManager().getStoragePolicySuite(), counts2, true);
+    QuotaCounts counts2 = sub1Node.computeQuotaUsage(
+        fsn.getBlockManager().getStoragePolicySuite(), true);
     assertEquals(sub1Node.dumpTreeRecursively().toString(), 0,
         counts2.getTypeSpaces().get(StorageType.SSD));
 
@@ -620,7 +619,7 @@ public class TestQuotaByStorageType {
     assertEquals(file1Len, ssdConsumed);
 
     // Truncate file to 1 * BLOCKSIZE
-    int newFile1Len = BLOCKSIZE * 1;
+    int newFile1Len = BLOCKSIZE;
     dfs.truncate(createdFile1, newFile1Len);
 
     // Verify SSD consumed after truncate

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

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import com.google.common.collect.Lists;
-import junit.framework.Assert;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
@@ -26,6 +25,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
+import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 
@@ -64,7 +64,8 @@ public class TestFileWithSnapshotFeature {
     ArrayList<INode> removedINodes = new ArrayList<>();
     INode.ReclaimContext ctx = new INode.ReclaimContext(
         bsps, collectedBlocks, removedINodes, null);
-    QuotaCounts counts = sf.updateQuotaAndCollectBlocks(ctx, file, diff);
+    sf.updateQuotaAndCollectBlocks(ctx, file, diff);
+    QuotaCounts counts = ctx.quotaDelta().getCountsCopy();
     Assert.assertEquals(0, counts.getStorageSpace());
     Assert.assertTrue(counts.getTypeSpaces().allLessOrEqual(0));
 
@@ -79,7 +80,8 @@ public class TestFileWithSnapshotFeature {
         .thenReturn(Lists.newArrayList(SSD));
     when(bsp.chooseStorageTypes(REPL_3))
         .thenReturn(Lists.newArrayList(DISK));
-    counts = sf.updateQuotaAndCollectBlocks(ctx, file, diff);
+    sf.updateQuotaAndCollectBlocks(ctx, file, diff);
+    counts = ctx.quotaDelta().getCountsCopy();
     Assert.assertEquals((REPL_3 - REPL_1) * BLOCK_SIZE,
                         counts.getStorageSpace());
     Assert.assertEquals(BLOCK_SIZE, counts.getTypeSpaces().get(DISK));

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

@@ -784,6 +784,7 @@ public class TestRenameWithSnapshots {
     
     // delete foo
     hdfs.delete(foo_dir1, true);
+    restartClusterAndCheckImage(true);
     hdfs.delete(bar2_dir1, true);
     
     // restart the cluster and check fsimage

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

@@ -208,8 +208,7 @@ public class TestSnapshotDeletion {
         q.getNameSpace());
     assertEquals(dirNode.dumpTreeRecursively().toString(), expectedDs,
         q.getStorageSpace());
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-    dirNode.computeQuotaUsage(fsdir.getBlockStoragePolicySuite(), counts, false);
+    QuotaCounts counts = dirNode.computeQuotaUsage(fsdir.getBlockStoragePolicySuite(), false);
     assertEquals(dirNode.dumpTreeRecursively().toString(), expectedNs,
         counts.getNameSpace());
     assertEquals(dirNode.dumpTreeRecursively().toString(), expectedDs,
@@ -248,8 +247,11 @@ public class TestSnapshotDeletion {
     DFSTestUtil.createFile(hdfs, metaChangeFile2, BLOCKSIZE, REPLICATION, seed);
     
     // Case 1: delete deleteDir before taking snapshots
+    hdfs.setQuota(dir, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
+    checkQuotaUsageComputation(dir, 10, BLOCKSIZE * REPLICATION * 4);
     hdfs.delete(deleteDir, true);
-    
+    checkQuotaUsageComputation(dir, 8, BLOCKSIZE * REPLICATION * 3);
+
     // create snapshot s0
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
     
@@ -542,7 +544,7 @@ public class TestSnapshotDeletion {
     
     // check 4: no snapshot copy for toDeleteFile
     try {
-      status = hdfs.getFileStatus(toDeleteFile);
+      hdfs.getFileStatus(toDeleteFile);
       fail("should throw FileNotFoundException");
     } catch (FileNotFoundException e) {
       GenericTestUtils.assertExceptionContains("File does not exist: "
@@ -552,7 +554,7 @@ public class TestSnapshotDeletion {
     final Path toDeleteFileInSnapshot = SnapshotTestHelper.getSnapshotPath(dir,
         "s0", toDeleteFile.toString().substring(dir.toString().length()));
     try {
-      status = hdfs.getFileStatus(toDeleteFileInSnapshot);
+      hdfs.getFileStatus(toDeleteFileInSnapshot);
       fail("should throw FileNotFoundException");
     } catch (FileNotFoundException e) {
       GenericTestUtils.assertExceptionContains("File does not exist: "

+ 1 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java

@@ -23,8 +23,6 @@ import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 
-import java.util.ArrayList;
-
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -75,8 +73,7 @@ public class TestSnapshotManager {
 
     // Delete a snapshot to free up a slot.
     //
-    sm.deleteSnapshot(iip, "", mock(INode.BlocksMapUpdateInfo.class),
-        new ArrayList<INode>());
+    sm.deleteSnapshot(iip, "", mock(INode.ReclaimContext.class));
 
     // Attempt to create a snapshot again. It should still fail due
     // to snapshot ID rollover.