You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zj...@apache.org on 2015/05/12 22:49:30 UTC

[13/36] hadoop git commit: HDFS-8357. Consolidate parameters of INode.CleanSubtree() into a parameter objects. Contributed by Li Lu.

HDFS-8357. Consolidate parameters of INode.CleanSubtree() into a parameter objects. Contributed by Li Lu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/51812970
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/51812970
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/51812970

Branch: refs/heads/YARN-2928
Commit: 5181297087f71d3978d87d8cec542c0a3ae6c5e9
Parents: 86ff073
Author: Haohui Mai <wh...@apache.org>
Authored: Sat May 9 22:51:58 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Tue May 12 13:24:12 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/FSDirDeleteOp.java     |  11 +-
 .../hdfs/server/namenode/FSDirRenameOp.java     |  12 +-
 .../hadoop/hdfs/server/namenode/INode.java      |  89 ++++++++------
 .../hdfs/server/namenode/INodeDirectory.java    |  44 +++----
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  35 +++---
 .../hadoop/hdfs/server/namenode/INodeMap.java   |   9 +-
 .../hdfs/server/namenode/INodeReference.java    |  59 ++++-----
 .../hdfs/server/namenode/INodeSymlink.java      |  15 +--
 .../namenode/snapshot/AbstractINodeDiff.java    |  15 +--
 .../snapshot/AbstractINodeDiffList.java         |  20 ++-
 .../snapshot/DirectorySnapshottableFeature.java |  12 +-
 .../snapshot/DirectoryWithSnapshotFeature.java  | 121 ++++++++-----------
 .../hdfs/server/namenode/snapshot/FileDiff.java |  18 ++-
 .../server/namenode/snapshot/FileDiffList.java  |  11 +-
 .../snapshot/FileWithSnapshotFeature.java       |  35 +++---
 .../namenode/snapshot/SnapshotManager.java      |   6 +-
 .../snapshot/TestFileWithSnapshotFeature.java   |   8 +-
 18 files changed, 232 insertions(+), 291 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 88503fb..f733f22 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -537,6 +537,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8327. Simplify quota calculations for snapshots and truncate. (wheat9)
 
+    HDFS-8357. Consolidate parameters of INode.CleanSubtree() into a parameter
+    objects. (Li Lu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
index c31d75f..f99e50c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
@@ -250,13 +250,14 @@ class FSDirDeleteOp {
     }
 
     // collect block and update quota
+    INode.ReclaimContext reclaimContext = new INode.ReclaimContext(
+        fsd.getBlockStoragePolicySuite(), collectedBlocks,
+        removedINodes, removedUCFiles);
     if (!targetNode.isInLatestSnapshot(latestSnapshot)) {
-      targetNode.destroyAndCollectBlocks(fsd.getBlockStoragePolicySuite(),
-        collectedBlocks, removedINodes, removedUCFiles);
+      targetNode.destroyAndCollectBlocks(reclaimContext);
     } else {
-      QuotaCounts counts = targetNode.cleanSubtree(
-        fsd.getBlockStoragePolicySuite(), CURRENT_STATE_ID,
-          latestSnapshot, collectedBlocks, removedINodes, removedUCFiles);
+      QuotaCounts counts = targetNode.cleanSubtree(reclaimContext,
+          CURRENT_STATE_ID, latestSnapshot);
       removed = counts.getNameSpace();
       fsd.updateCountNoQuotaCheck(iip, iip.length() -1, counts.negation());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
index d5faa78..7675907 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
@@ -732,14 +732,16 @@ class FSDirRenameOp {
       List<Long> removedUCFiles = new ChunkedArrayList<>();
       final boolean filesDeleted;
       if (!oldDstChild.isInLatestSnapshot(dstIIP.getLatestSnapshotId())) {
-        oldDstChild.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
-                                            removedUCFiles);
+        oldDstChild.destroyAndCollectBlocks(
+            new INode.ReclaimContext(bsps, collectedBlocks, removedINodes, removedUCFiles));
         filesDeleted = true;
       } else {
         filesDeleted = oldDstChild.cleanSubtree(
-            bsps, Snapshot.CURRENT_STATE_ID,
-            dstIIP.getLatestSnapshotId(), collectedBlocks,
-            removedINodes, removedUCFiles).getNameSpace() >= 0;
+            new INode.ReclaimContext(bsps, collectedBlocks, removedINodes,
+                                     removedUCFiles),
+            Snapshot.CURRENT_STATE_ID,
+            dstIIP.getLatestSnapshotId())
+            .getNameSpace() >= 0;
       }
       fsd.getFSNamesystem().removeLeasesAndINodes(
           removedUCFiles, removedINodes, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index b65879f..64af76f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -387,30 +387,20 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * snapshot in its diff list. Recursively clean its children.
    * </pre>
    *
-   * @param bsps
-   *          block storage policy suite to calculate intended storage type usage
+   * @param reclaimContext
+   *        Record blocks and inodes that need to be reclaimed.
    * @param snapshotId
-   *          The id of the snapshot to delete.
-   *          {@link Snapshot#CURRENT_STATE_ID} means to delete the current
-   *          file/directory.
+   *        The id of the snapshot to delete.
+   *        {@link Snapshot#CURRENT_STATE_ID} means to delete the current
+   *        file/directory.
    * @param priorSnapshotId
-   *          The id of the latest snapshot before the to-be-deleted snapshot.
-   *          When deleting a current inode, this parameter captures the latest
-   *          snapshot.
-   * @param collectedBlocks
-   *          blocks collected from the descents for further block
-   *          deletion/update will be added to the given map.
-   * @param removedINodes
-   *          INodes collected from the descents for further cleaning up of
-   *          inodeMap
-   * @param removedUCFiles
-   *          INodes whose leases need to be released
+   *        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(
-      final BlockStoragePolicySuite bsps, final int snapshotId,
-      int priorSnapshotId, BlocksMapUpdateInfo collectedBlocks,
-      List<INode> removedINodes, List<Long> removedUCFiles);
+      ReclaimContext reclaimContext, final int snapshotId, int priorSnapshotId);
   
   /**
    * Destroy self and clear everything! If the INode is a file, this method
@@ -418,22 +408,11 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * directory, the method goes down the subtree and collects blocks from the
    * descents, and clears its parent/children references as well. The method
    * also clears the diff list if the INode contains snapshot diff list.
-   * @param bsps
-   *          block storage policy suite to calculate intended storage type usage
-   *          This is needed because INodeReference#destroyAndCollectBlocks() needs
-   *          to call INode#cleanSubtree(), which calls INode#computeQuotaUsage().
-   * @param collectedBlocks
-   *          blocks collected from the descents for further block
-   *          deletion/update will be added to this map.
-   * @param removedINodes
-   *          INodes collected from the descents for further cleaning up of
-   *          inodeMap
-   * @param removedUCFiles
-   *          INodes whose leases need to be released
-   */
-  public abstract void destroyAndCollectBlocks(
-      BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
-      List<INode> removedINodes, List<Long> removedUCFiles);
+   *
+   * @param reclaimContext
+   *        Record blocks and inodes that need to be reclaimed.
+   */
+  public abstract void destroyAndCollectBlocks(ReclaimContext reclaimContext);
 
   /** Compute {@link ContentSummary}. Blocking call */
   public final ContentSummary computeContentSummary(BlockStoragePolicySuite bsps) {
@@ -823,7 +802,45 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     out.print(getParentString());
     out.print(", " + getPermissionStatus(snapshotId));
   }
-  
+
+  /**
+   * Context object to record blocks and inodes that need to be reclaimed
+   */
+  public static class ReclaimContext {
+    protected final BlockStoragePolicySuite bsps;
+    protected final BlocksMapUpdateInfo collectedBlocks;
+    protected final List<INode> removedINodes;
+    protected final List<Long> removedUCFiles;
+    /**
+     * @param bsps
+     *          block storage policy suite to calculate intended storage type
+     *          usage
+     * @param collectedBlocks
+     *          blocks collected from the descents for further block
+     *          deletion/update will be added to the given map.
+     * @param removedINodes
+ *          INodes collected from the descents for further cleaning up of
+     * @param removedUCFiles
+     *      files that the NN need to remove the leases
+     */
+    public ReclaimContext(
+        BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
+        List<INode> removedINodes, List<Long> removedUCFiles) {
+      this.bsps = bsps;
+      this.collectedBlocks = collectedBlocks;
+      this.removedINodes = removedINodes;
+      this.removedUCFiles = removedUCFiles;
+    }
+
+    public BlockStoragePolicySuite storagePolicySuite() {
+      return bsps;
+    }
+
+    public BlocksMapUpdateInfo collectedBlocks() {
+      return collectedBlocks;
+    }
+  }
+
   /**
    * Information used for updating the blocksMap when deleting files.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
index fa63889..106c9f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
@@ -264,11 +264,11 @@ public class INodeDirectory extends INodeWithAdditionalFields
     return getDirectorySnapshottableFeature().addSnapshot(this, id, name);
   }
 
-  public Snapshot removeSnapshot(BlockStoragePolicySuite bsps, String snapshotName,
-      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
+  public Snapshot removeSnapshot(
+      ReclaimContext reclaimContext, String snapshotName)
       throws SnapshotException {
-    return getDirectorySnapshottableFeature().removeSnapshot(bsps, this,
-        snapshotName, collectedBlocks, removedINodes);
+    return getDirectorySnapshottableFeature().removeSnapshot(
+        reclaimContext, this, snapshotName);
   }
 
   public void renameSnapshot(String path, String oldName, String newName)
@@ -754,9 +754,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
 
   /** Call cleanSubtree(..) recursively down the subtree. */
   public QuotaCounts cleanSubtreeRecursively(
-      final BlockStoragePolicySuite bsps, final int snapshot, int prior,
-      final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, List<Long> removedUCFiles,
+      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
@@ -771,8 +769,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
           && excludedNodes.containsKey(child)) {
         continue;
       } else {
-        QuotaCounts childCounts = child.cleanSubtree(bsps, snapshot, prior,
-            collectedBlocks, removedINodes, removedUCFiles);
+        QuotaCounts childCounts = child.cleanSubtree(reclaimContext, snapshot, prior);
         counts.add(childCounts);
       }
     }
@@ -780,49 +777,42 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   @Override
-  public void destroyAndCollectBlocks(
-      final BlockStoragePolicySuite bsps,
-      final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, List<Long> removedUCFiles) {
+  public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
     final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     if (sf != null) {
-      sf.clear(bsps, this, collectedBlocks, removedINodes, removedUCFiles);
+      sf.clear(reclaimContext, this);
     }
     for (INode child : getChildrenList(Snapshot.CURRENT_STATE_ID)) {
-      child.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
-                                    removedUCFiles);
+      child.destroyAndCollectBlocks(reclaimContext);
     }
     if (getAclFeature() != null) {
       AclStorage.removeAclFeature(getAclFeature());
     }
     clear();
-    removedINodes.add(this);
+    reclaimContext.removedINodes.add(this);
   }
   
   @Override
   public QuotaCounts cleanSubtree(
-      final BlockStoragePolicySuite bsps, final int snapshotId, int priorSnapshotId,
-      final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, List<Long> removedUCFiles) {
+      ReclaimContext reclaimContext, final int snapshotId, int priorSnapshotId) {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     // there is snapshot data
     if (sf != null) {
-      return sf.cleanDirectory(bsps, this, snapshotId, priorSnapshotId,
-          collectedBlocks, removedINodes, removedUCFiles);
+      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(bsps, counts, true);
-      destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
-                              removedUCFiles);
+      this.computeQuotaUsage(reclaimContext.bsps, counts, true);
+      destroyAndCollectBlocks(reclaimContext);
       return counts; 
     } else {
       // process recursively down the subtree
-      QuotaCounts counts = cleanSubtreeRecursively(bsps, snapshotId, priorSnapshotId,
-          collectedBlocks, removedINodes, removedUCFiles, null);
+      QuotaCounts counts = cleanSubtreeRecursively(
+          reclaimContext, snapshotId, priorSnapshotId, null);
       if (isQuotaSet()) {
         getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(counts.negation());
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 14fc7b0..3790c74 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -492,13 +492,10 @@ public class INodeFile extends INodeWithAdditionalFields
 
   @Override
   public QuotaCounts cleanSubtree(
-      BlockStoragePolicySuite bsps, final int snapshot, int priorSnapshotId,
-      final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, List<Long> removedUCFiles) {
+      ReclaimContext reclaimContext, final int snapshot, int priorSnapshotId) {
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf != null) {
-      return sf.cleanFile(bsps, this, snapshot, priorSnapshotId, collectedBlocks,
-          removedINodes);
+      return sf.cleanFile(reclaimContext, this, snapshot, priorSnapshotId);
     }
     QuotaCounts counts = new QuotaCounts.Builder().build();
 
@@ -506,17 +503,16 @@ public class INodeFile extends INodeWithAdditionalFields
       if (priorSnapshotId == NO_SNAPSHOT_ID) {
         // this only happens when deleting the current file and the file is not
         // in any snapshot
-        computeQuotaUsage(bsps, counts, false);
-        destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
-                                removedUCFiles);
+        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, collectedBlocks);
-          if (removedUCFiles != null) {
-            removedUCFiles.add(getId());
+          uc.cleanZeroSizeBlock(this, reclaimContext.collectedBlocks);
+          if (reclaimContext.removedUCFiles != null) {
+            reclaimContext.removedUCFiles.add(getId());
           }
         }
       }
@@ -525,12 +521,10 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override
-  public void destroyAndCollectBlocks(
-      BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, List<Long> removedUCFiles) {
-    if (blocks != null && collectedBlocks != null) {
+  public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
+    if (blocks != null && reclaimContext.collectedBlocks != null) {
       for (BlockInfoContiguous blk : blocks) {
-        collectedBlocks.addDeleteBlock(blk);
+        reclaimContext.collectedBlocks.addDeleteBlock(blk);
         blk.setBlockCollection(null);
       }
     }
@@ -539,14 +533,15 @@ public class INodeFile extends INodeWithAdditionalFields
       AclStorage.removeAclFeature(getAclFeature());
     }
     clear();
-    removedINodes.add(this);
+    reclaimContext.removedINodes.add(this);
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf != null) {
-      sf.getDiffs().destroyAndCollectSnapshotBlocks(collectedBlocks);
+      sf.getDiffs().destroyAndCollectSnapshotBlocks(
+          reclaimContext.collectedBlocks);
       sf.clearDiffs();
     }
-    if (isUnderConstruction() && removedUCFiles != null) {
-      removedUCFiles.add(getId());
+    if (isUnderConstruction() && reclaimContext.removedUCFiles != null) {
+      reclaimContext.removedUCFiles.add(getId());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
index 5f16bd6..e1cb869 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.Iterator;
-import java.util.List;
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -97,9 +96,7 @@ public class INodeMap {
       }
       
       @Override
-      public void destroyAndCollectBlocks(
-          BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
-          List<INode> removedINodes, List<Long> removedUCFiles) {
+      public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
         // Nothing to do
       }
 
@@ -118,9 +115,7 @@ public class INodeMap {
       
       @Override
       public QuotaCounts cleanSubtree(
-          BlockStoragePolicySuite bsps, int snapshotId, int priorSnapshotId,
-          BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes,
-          List<Long> removedUCFiles) {
+          ReclaimContext reclaimContext, int snapshotId, int priorSnapshotId) {
           return null;
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
index 5008dc0..90a2eb1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
@@ -303,19 +303,15 @@ public abstract class INodeReference extends INode {
 
   @Override // used by WithCount
   public QuotaCounts cleanSubtree(
-      BlockStoragePolicySuite bsps, int snapshot, int prior, BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, List<Long> removedUCFiles) {
-    return referred.cleanSubtree(bsps, snapshot, prior, collectedBlocks,
-        removedINodes, removedUCFiles);
+      ReclaimContext reclaimContext, int snapshot, int prior) {
+    return referred.cleanSubtree(reclaimContext,
+                                 snapshot, prior);
   }
 
   @Override // used by WithCount
-  public void destroyAndCollectBlocks(
-      BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, List<Long> removedUCFiles) {
+  public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
     if (removeReference(this) <= 0) {
-      referred.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
-                                       removedUCFiles);
+      referred.destroyAndCollectBlocks(reclaimContext);
     }
   }
 
@@ -544,8 +540,7 @@ public abstract class INodeReference extends INode {
     
     @Override
     public QuotaCounts cleanSubtree(
-        BlockStoragePolicySuite bsps, final int snapshot, int prior, final BlocksMapUpdateInfo collectedBlocks,
-        final List<INode> removedINodes, List<Long> removedUCFiles) {
+        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);
@@ -560,8 +555,8 @@ public abstract class INodeReference extends INode {
         return new QuotaCounts.Builder().build();
       }
 
-      QuotaCounts counts = getReferredINode().cleanSubtree(bsps, snapshot, prior,
-          collectedBlocks, removedINodes, removedUCFiles);
+      QuotaCounts counts = getReferredINode().cleanSubtree(reclaimContext,
+          snapshot, prior);
       INodeReference ref = getReferredINode().getParentReference();
       if (ref != null) {
         try {
@@ -582,13 +577,10 @@ public abstract class INodeReference extends INode {
     }
     
     @Override
-    public void destroyAndCollectBlocks(
-        BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
-        final List<INode> removedINodes, List<Long> removedUCFiles) {
+    public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
       int snapshot = getSelfSnapshot();
       if (removeReference(this) <= 0) {
-        getReferredINode().destroyAndCollectBlocks(bsps, collectedBlocks,
-            removedINodes, removedUCFiles);
+        getReferredINode().destroyAndCollectBlocks(reclaimContext);
       } else {
         int prior = getPriorSnapshot(this);
         INode referred = getReferredINode().asReference().getReferredINode();
@@ -607,8 +599,8 @@ public abstract class INodeReference extends INode {
             return;
           }
           try {
-            QuotaCounts counts = referred.cleanSubtree(bsps, snapshot, prior,
-                collectedBlocks, removedINodes, removedUCFiles);
+            QuotaCounts counts = referred.cleanSubtree(reclaimContext,
+                snapshot, prior);
             INodeReference ref = getReferredINode().getParentReference();
             if (ref != null) {
               ref.addSpaceConsumed(counts.negation(), true);
@@ -663,15 +655,12 @@ public abstract class INodeReference extends INode {
     
     @Override
     public QuotaCounts cleanSubtree(
-        BlockStoragePolicySuite bsps, int snapshot, int prior,
-        BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes,
-        List<Long> removedUCFiles) {
+        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(bsps, counts, true);
-        destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
-                                removedUCFiles);
+        this.computeQuotaUsage(reclaimContext.bsps, counts, true);
+        destroyAndCollectBlocks(reclaimContext);
         return counts;
       } else {
         // if prior is NO_SNAPSHOT_ID, we need to check snapshot belonging to 
@@ -687,8 +676,7 @@ public abstract class INodeReference extends INode {
             && Snapshot.ID_INTEGER_COMPARATOR.compare(snapshot, prior) <= 0) {
           return new QuotaCounts.Builder().build();
         }
-        return getReferredINode().cleanSubtree(bsps, snapshot, prior,
-            collectedBlocks, removedINodes, removedUCFiles);
+        return getReferredINode().cleanSubtree(reclaimContext, snapshot, prior);
       }
     }
     
@@ -701,14 +689,12 @@ public abstract class INodeReference extends INode {
      * referred node's subtree and delete everything created after the last 
      * rename operation, i.e., everything outside of the scope of the prior 
      * WithName nodes.
+     * @param reclaimContext
      */
     @Override
-    public void destroyAndCollectBlocks(
-        BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
-        final List<INode> removedINodes, List<Long> removedUCFiles) {
+    public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
       if (removeReference(this) <= 0) {
-        getReferredINode().destroyAndCollectBlocks(bsps, collectedBlocks,
-            removedINodes, removedUCFiles);
+        getReferredINode().destroyAndCollectBlocks(reclaimContext);
       } else {
         // we will clean everything, including files, directories, and 
         // snapshots, that were created after this prior snapshot
@@ -730,16 +716,15 @@ 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(bsps, snapshot, prior, collectedBlocks,
-              removedINodes, removedUCFiles);
+          referred.cleanSubtree(reclaimContext, 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(bsps, dir, snapshot,
-                prior, collectedBlocks, removedINodes, removedUCFiles);
+            DirectoryWithSnapshotFeature.destroyDstSubtree(
+                reclaimContext, dir, snapshot, prior);
           } catch (QuotaExceededException e) {
             LOG.error("should not exceed quota while snapshot deletion", e);
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
index 7ce893f..ad696b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.PrintWriter;
-import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -74,23 +73,17 @@ public class INodeSymlink extends INodeWithAdditionalFields {
   
   @Override
   public QuotaCounts cleanSubtree(
-      BlockStoragePolicySuite bsps, final int snapshotId, int priorSnapshotId,
-      final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, List<Long> removedUCFiles) {
+      ReclaimContext reclaimContext, final int snapshotId, int priorSnapshotId) {
     if (snapshotId == Snapshot.CURRENT_STATE_ID
         && priorSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
-      destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
-                              removedUCFiles);
+      destroyAndCollectBlocks(reclaimContext);
     }
     return new QuotaCounts.Builder().nameSpace(1).build();
   }
   
   @Override
-  public void destroyAndCollectBlocks(
-      final BlockStoragePolicySuite bsps,
-      final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, List<Long> removedUCFiles) {
-    removedINodes.add(this);
+  public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
+    reclaimContext.removedINodes.add(this);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
index 691d717..89d3625 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
@@ -116,21 +116,18 @@ abstract class AbstractINodeDiff<N extends INode,
 
   /** Combine the posterior diff and collect blocks for deletion. */
   abstract QuotaCounts combinePosteriorAndCollectBlocks(
-      final BlockStoragePolicySuite bsps, final N currentINode,
-      final D posterior, final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes);
+      INode.ReclaimContext reclaimContext, final N currentINode,
+      final D posterior);
   
   /**
    * Delete and clear self.
-   * @param bsps The block storage policy suite used to retrieve storage policy
+   * @param reclaimContext blocks and inodes that need to be reclaimed
    * @param currentINode The inode where the deletion happens.
-   * @param collectedBlocks Used to collect blocks for deletion.
-   * @param removedINodes INodes removed
-   * @return quota usage delta
+   *
+   * @return usage delta
    */
   abstract QuotaCounts destroyDiffAndCollectBlocks(
-      final BlockStoragePolicySuite bsps, final N currentINode,
-      final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes);
+      INode.ReclaimContext reclaimContext, final N currentINode);
 
   @Override
   public String toString() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
index fb13e09..43127fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
@@ -22,9 +22,7 @@ import java.util.Collections;
 import java.util.Iterator;
 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;
 
@@ -62,16 +60,14 @@ abstract class AbstractINodeDiffList<N extends INode,
    * outside. If the diff to remove is not the first one in the diff list, we 
    * need to combine the diff with its previous one.
    * 
+   * @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
-   * @param collectedBlocks Used to collect information for blocksMap update
    * @return delta in namespace.
    */
-  public final QuotaCounts deleteSnapshotDiff(BlockStoragePolicySuite bsps,
-      final int snapshot,
-      final int prior, final N currentINode,
-      final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes) {
+  public final QuotaCounts 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();
@@ -82,8 +78,8 @@ abstract class AbstractINodeDiffList<N extends INode,
         diffs.get(snapshotIndex).setSnapshotId(prior);
       } else { // there is no snapshot before
         removed = diffs.remove(0);
-        counts.add(removed.destroyDiffAndCollectBlocks(bsps, currentINode,
-            collectedBlocks, removedINodes));
+        counts.add(removed.destroyDiffAndCollectBlocks(reclaimContext,
+            currentINode));
       }
     } else if (snapshotIndex > 0) {
       final AbstractINodeDiff<N, A, D> previous = diffs.get(snapshotIndex - 1);
@@ -96,8 +92,8 @@ abstract class AbstractINodeDiffList<N extends INode,
           previous.snapshotINode = removed.snapshotINode;
         }
 
-        counts.add(previous.combinePosteriorAndCollectBlocks(
-            bsps, currentINode, removed, collectedBlocks, removedINodes));
+        counts.add(previous.combinePosteriorAndCollectBlocks(reclaimContext,
+            currentINode, removed));
         previous.setPosterior(removed.getPosterior());
         removed.setPosterior(null);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
index dc58856..504a64a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
@@ -197,15 +197,15 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
    * Remove the snapshot with the given name from {@link #snapshotsByNames},
    * and delete all the corresponding DirectoryDiff.
    *
+   * @param reclaimContext records blocks and inodes that need to be reclaimed
    * @param snapshotRoot The directory where we take snapshots
    * @param snapshotName The name of the snapshot to be removed
-   * @param collectedBlocks Used to collect information to update blocksMap
    * @return The removed snapshot. Null if no snapshot with the given name
    *         exists.
    */
-  public Snapshot removeSnapshot(BlockStoragePolicySuite bsps, INodeDirectory snapshotRoot,
-      String snapshotName, BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes) throws SnapshotException {
+  public Snapshot removeSnapshot(
+      INode.ReclaimContext reclaimContext, INodeDirectory snapshotRoot,
+      String snapshotName) throws SnapshotException {
     final int i = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
     if (i < 0) {
       throw new SnapshotException("Cannot delete snapshot " + snapshotName
@@ -215,8 +215,8 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
       final Snapshot snapshot = snapshotsByNames.get(i);
       int prior = Snapshot.findLatestSnapshot(snapshotRoot, snapshot.getId());
       try {
-        QuotaCounts counts = snapshotRoot.cleanSubtree(bsps, snapshot.getId(),
-            prior, collectedBlocks, removedINodes, null);
+        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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
index bd2dc2d..adb012a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.server.namenode.ContentCounts;
 import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 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.INodeDirectoryAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@@ -97,15 +96,12 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
 
     /** clear the created list */
     private QuotaCounts destroyCreatedList(
-        final BlockStoragePolicySuite bsps, final INodeDirectory currentINode,
-        final BlocksMapUpdateInfo collectedBlocks,
-        final List<INode> removedINodes, List<Long> removedUCFiles) {
+        INode.ReclaimContext reclaimContext, final INodeDirectory currentINode) {
       QuotaCounts counts = new QuotaCounts.Builder().build();
       final List<INode> createdList = getList(ListType.CREATED);
       for (INode c : createdList) {
-        c.computeQuotaUsage(bsps, counts, true);
-        c.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
-                                  removedUCFiles);
+        c.computeQuotaUsage(reclaimContext.storagePolicySuite(), counts, true);
+        c.destroyAndCollectBlocks(reclaimContext);
         // c should be contained in the children list, remove it
         currentINode.removeChild(c);
       }
@@ -114,16 +110,12 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     /** clear the deleted list */
-    private QuotaCounts destroyDeletedList(
-        final BlockStoragePolicySuite bsps,
-        final BlocksMapUpdateInfo collectedBlocks,
-        final List<INode> removedINodes, List<Long> removedUCFiles) {
+    private QuotaCounts destroyDeletedList(INode.ReclaimContext reclaimContext) {
       QuotaCounts counts = new QuotaCounts.Builder().build();
       final List<INode> deletedList = getList(ListType.DELETED);
       for (INode d : deletedList) {
-        d.computeQuotaUsage(bsps, counts, false);
-        d.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
-                                  removedUCFiles);
+        d.computeQuotaUsage(reclaimContext.storagePolicySuite(), counts, false);
+        d.destroyAndCollectBlocks(reclaimContext);
       }
       deletedList.clear();
       return counts;
@@ -211,19 +203,17 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
 
     @Override
     QuotaCounts combinePosteriorAndCollectBlocks(
-        final BlockStoragePolicySuite bsps, final INodeDirectory currentDir,
-        final DirectoryDiff posterior,
-        final BlocksMapUpdateInfo collectedBlocks,
-        final List<INode> removedINodes) {
+        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(bsps, counts, false);
-            inode.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
-                                          null);
+            inode.computeQuotaUsage(reclaimContext.storagePolicySuite(), counts, false);
+            inode.destroyAndCollectBlocks(reclaimContext);
           }
         }
       });
@@ -322,12 +312,10 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
 
     @Override
     QuotaCounts destroyDiffAndCollectBlocks(
-        BlockStoragePolicySuite bsps, INodeDirectory currentINode,
-        BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
+        INode.ReclaimContext reclaimContext, INodeDirectory currentINode) {
       // this diff has been deleted
       QuotaCounts counts = new QuotaCounts.Builder().build();
-      counts.add(diff.destroyDeletedList(bsps, collectedBlocks, removedINodes,
-                                         null));
+      counts.add(diff.destroyDeletedList(reclaimContext));
       INodeDirectoryAttributes snapshotINode = getSnapshotINode();
       if (snapshotINode != null && snapshotINode.getAclFeature() != null) {
         AclStorage.removeAclFeature(snapshotINode.getAclFeature());
@@ -412,25 +400,24 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
    * Destroy a subtree under a DstReference node.
    */
   public static void destroyDstSubtree(
-      final BlockStoragePolicySuite bsps, INode inode, final int snapshot,
-      final int prior, final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, List<Long> removedUCFiles) throws QuotaExceededException {
+      INode.ReclaimContext reclaimContext, INode inode, final int snapshot,
+      final int prior) throws QuotaExceededException {
     Preconditions.checkArgument(prior != Snapshot.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(bsps, snapshot, prior, collectedBlocks, removedINodes,
-                           removedUCFiles);
+        inode.cleanSubtree(reclaimContext,
+            snapshot, prior);
       } else { 
         // for DstReference node, continue this process to its subtree
-        destroyDstSubtree(bsps, inode.asReference().getReferredINode(), snapshot,
-            prior, collectedBlocks, removedINodes, removedUCFiles);
+        destroyDstSubtree(reclaimContext,
+                          inode.asReference().getReferredINode(), snapshot,
+                          prior);
       }
     } else if (inode.isFile()) {
-      inode.cleanSubtree(bsps, snapshot, prior, collectedBlocks, removedINodes,
-                         removedUCFiles);
+      inode.cleanSubtree(reclaimContext, snapshot, prior);
     } else if (inode.isDirectory()) {
       Map<INode, INode> excludedNodes = null;
       INodeDirectory dir = inode.asDirectory();
@@ -444,21 +431,19 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         }
         
         if (snapshot != Snapshot.CURRENT_STATE_ID) {
-          diffList.deleteSnapshotDiff(bsps, snapshot, prior, dir, collectedBlocks,
-              removedINodes);
+          diffList.deleteSnapshotDiff(reclaimContext,
+              snapshot, prior, dir);
         }
         priorDiff = diffList.getDiffById(prior);
         if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
-          priorDiff.diff.destroyCreatedList(bsps, dir, collectedBlocks,
-              removedINodes, removedUCFiles);
+          priorDiff.diff.destroyCreatedList(reclaimContext, dir);
         }
       }
       for (INode child : inode.asDirectory().getChildrenList(prior)) {
         if (excludedNodes != null && excludedNodes.containsKey(child)) {
           continue;
         }
-        destroyDstSubtree(bsps, child, snapshot, prior, collectedBlocks,
-            removedINodes, removedUCFiles);
+        destroyDstSubtree(reclaimContext, child, snapshot, prior);
       }
     }
   }
@@ -466,18 +451,15 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
   /**
    * Clean an inode while we move it from the deleted list of post to the
    * deleted list of prior.
-   * @param bsps The block storage policy suite.
+   * @param reclaimContext blocks and inodes that need to be reclaimed
    * @param inode The inode to clean.
    * @param post The post snapshot.
    * @param prior The id of the prior snapshot.
-   * @param collectedBlocks Used to collect blocks for later deletion.
-   * @param removedUCFiles
    * @return Quota usage update.
    */
   private static QuotaCounts cleanDeletedINode(
-      final BlockStoragePolicySuite bsps, INode inode, final int post, final int prior,
-      final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, List<Long> removedUCFiles) {
+      INode.ReclaimContext reclaimContext, INode inode, final int post,
+      final int prior) {
     QuotaCounts counts = new QuotaCounts.Builder().build();
     Deque<INode> queue = new ArrayDeque<INode>();
     queue.addLast(inode);
@@ -486,15 +468,13 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
       if (topNode instanceof INodeReference.WithName) {
         INodeReference.WithName wn = (INodeReference.WithName) topNode;
         if (wn.getLastSnapshotId() >= post) {
-          wn.cleanSubtree(bsps, post, prior, collectedBlocks, removedINodes,
-                          removedUCFiles);
+          wn.cleanSubtree(reclaimContext, post, prior);
         }
         // For DstReference node, since the node is not in the created list of
         // 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(bsps, post, prior, file,
-            collectedBlocks, removedINodes));
+        counts.add(file.getDiffs().deleteSnapshotDiff(reclaimContext, post, prior, file));
       } else if (topNode.isDirectory()) {
         INodeDirectory dir = topNode.asDirectory();
         ChildrenDiff priorChildrenDiff = null;
@@ -505,8 +485,8 @@ 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(bsps, dir,
-                collectedBlocks, removedINodes, removedUCFiles));
+            counts.add(priorChildrenDiff.destroyCreatedList(reclaimContext,
+                dir));
           }
         }
         
@@ -636,13 +616,11 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     return child;
   }
 
-  public void clear(BlockStoragePolicySuite bsps, INodeDirectory currentINode,
-      final BlocksMapUpdateInfo collectedBlocks, final List<INode>
-      removedINodes, final List<Long> removedUCFiles) {
+  public void clear(
+      INode.ReclaimContext reclaimContext, INodeDirectory currentINode) {
     // destroy its diff list
     for (DirectoryDiff diff : diffs) {
-      diff.destroyDiffAndCollectBlocks(bsps, currentINode, collectedBlocks,
-        removedINodes);
+      diff.destroyDiffAndCollectBlocks(reclaimContext, currentINode);
     }
     diffs.clear();
   }
@@ -729,9 +707,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
   }
 
   public QuotaCounts cleanDirectory(
-      final BlockStoragePolicySuite bsps, final INodeDirectory currentINode,
-      final int snapshot, int prior, final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, List<Long> removedUCFiles) {
+      INode.ReclaimContext reclaimContext, final INodeDirectory currentINode,
+      final int snapshot, int prior) {
     QuotaCounts counts = new QuotaCounts.Builder().build();
     Map<INode, INode> priorCreated = null;
     Map<INode, INode> priorDeleted = null;
@@ -740,11 +717,11 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
       // delete everything in created list
       DirectoryDiff lastDiff = diffs.getLast();
       if (lastDiff != null) {
-        counts.add(lastDiff.diff.destroyCreatedList(bsps, currentINode,
-            collectedBlocks, removedINodes, removedUCFiles));
+        counts.add(lastDiff.diff.destroyCreatedList(reclaimContext,
+            currentINode));
       }
-      counts.add(currentINode.cleanSubtreeRecursively(bsps, snapshot, prior,
-          collectedBlocks, removedINodes, removedUCFiles, priorDeleted));
+      counts.add(currentINode.cleanSubtreeRecursively(reclaimContext,
+          snapshot, prior, priorDeleted));
     } else {
       // update prior
       prior = getDiffs().updatePrior(snapshot, prior);
@@ -760,10 +737,10 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         }
       }
       
-      counts.add(getDiffs().deleteSnapshotDiff(bsps, snapshot, prior,
-          currentINode, collectedBlocks, removedINodes));
-      counts.add(currentINode.cleanSubtreeRecursively(bsps, snapshot, prior,
-          collectedBlocks, removedINodes, removedUCFiles, priorDeleted));
+      counts.add(getDiffs().deleteSnapshotDiff(reclaimContext, snapshot, prior,
+          currentINode));
+      counts.add(currentINode.cleanSubtreeRecursively(reclaimContext,
+          snapshot, prior, priorDeleted));
 
       // check priorDiff again since it may be created during the diff deletion
       if (prior != Snapshot.NO_SNAPSHOT_ID) {
@@ -779,8 +756,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
             for (INode cNode : priorDiff.getChildrenDiff().getList(
                 ListType.CREATED)) {
               if (priorCreated.containsKey(cNode)) {
-                counts.add(cNode.cleanSubtree(bsps, snapshot, Snapshot.NO_SNAPSHOT_ID,
-                    collectedBlocks, removedINodes, removedUCFiles));
+                counts.add(cNode.cleanSubtree(reclaimContext,
+                    snapshot, Snapshot.NO_SNAPSHOT_ID));
               }
             }
           }
@@ -796,8 +773,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           for (INode dNode : priorDiff.getChildrenDiff().getList(
               ListType.DELETED)) {
             if (priorDeleted == null || !priorDeleted.containsKey(dNode)) {
-              counts.add(cleanDeletedINode(bsps, dNode, snapshot, prior,
-                  collectedBlocks, removedINodes, removedUCFiles));
+              counts.add(cleanDeletedINode(reclaimContext,
+                  dNode, snapshot, prior));
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
index 931f7f0..dd8be82 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
@@ -20,10 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.List;
 
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -82,13 +80,12 @@ public class FileDiff extends
 
   @Override
   QuotaCounts combinePosteriorAndCollectBlocks(
-      BlockStoragePolicySuite bsps, INodeFile currentINode,
-      FileDiff posterior, BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes) {
+      INode.ReclaimContext reclaimContext, INodeFile currentINode,
+      FileDiff posterior) {
     FileWithSnapshotFeature sf = currentINode.getFileWithSnapshotFeature();
     assert sf != null : "FileWithSnapshotFeature is null";
-    return sf.updateQuotaAndCollectBlocks(
-        bsps, currentINode, posterior, collectedBlocks, removedINodes);
+    return sf.updateQuotaAndCollectBlocks(reclaimContext,
+        currentINode, posterior);
   }
   
   @Override
@@ -112,11 +109,10 @@ public class FileDiff extends
   }
 
   @Override
-  QuotaCounts destroyDiffAndCollectBlocks(BlockStoragePolicySuite bsps, INodeFile currentINode,
-      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
+  QuotaCounts destroyDiffAndCollectBlocks(
+      INode.ReclaimContext reclaimContext, INodeFile currentINode) {
     return currentINode.getFileWithSnapshotFeature()
-        .updateQuotaAndCollectBlocks(bsps, currentINode, this, collectedBlocks,
-            removedINodes);
+        .updateQuotaAndCollectBlocks(reclaimContext, currentINode, this);
   }
 
   public void destroyAndCollectSnapshotBlocks(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index 5c9e121..0788e75 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@ -23,7 +23,6 @@ import java.util.List;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -97,16 +96,14 @@ public class FileDiffList extends
    * up to the file length of the latter.
    * Collect unused blocks of the removed snapshot.
    */
-  void combineAndCollectSnapshotBlocks(BlockStoragePolicySuite bsps, INodeFile file,
-                                       FileDiff removed,
-                                       BlocksMapUpdateInfo collectedBlocks,
-                                       List<INode> removedINodes) {
+  void combineAndCollectSnapshotBlocks(
+      INode.ReclaimContext reclaimContext, INodeFile file, FileDiff removed) {
     BlockInfoContiguous[] removedBlocks = removed.getBlocks();
     if(removedBlocks == null) {
       FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
       assert sf != null : "FileWithSnapshotFeature is null";
       if(sf.isCurrentFileDeleted())
-        sf.collectBlocksAndClear(bsps, file, collectedBlocks, removedINodes);
+        sf.collectBlocksAndClear(reclaimContext, file);
       return;
     }
     int p = getPrior(removed.getSnapshotId(), true);
@@ -139,7 +136,7 @@ public class FileDiffList extends
     // Collect the remaining blocks of the file, ignoring truncate block
     for(;i < removedBlocks.length; i++) {
       if(dontRemoveBlock == null || !removedBlocks[i].equals(dontRemoveBlock)) {
-        collectedBlocks.addDeleteBlock(removedBlocks[i]);
+        reclaimContext.collectedBlocks().addDeleteBlock(removedBlocks[i]);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
index 7d884d3..3bb549b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
@@ -23,10 +23,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.AclStorage;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
@@ -118,22 +116,21 @@ public class FileWithSnapshotFeature implements INode.Feature {
     return (isCurrentFileDeleted()? "(DELETED), ": ", ") + diffs;
   }
   
-  public QuotaCounts cleanFile(final BlockStoragePolicySuite bsps,
+  public QuotaCounts cleanFile(INode.ReclaimContext reclaimContext,
       final INodeFile file, final int snapshotId,
-      int priorSnapshotId, final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes) {
+      int priorSnapshotId) {
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       // delete the current file while the file has snapshot feature
       if (!isCurrentFileDeleted()) {
         file.recordModification(priorSnapshotId);
         deleteCurrentFile();
       }
-      collectBlocksAndClear(bsps, file, collectedBlocks, removedINodes);
+      collectBlocksAndClear(reclaimContext, file);
       return new QuotaCounts.Builder().build();
     } else { // delete the snapshot
       priorSnapshotId = getDiffs().updatePrior(snapshotId, priorSnapshotId);
-      return diffs.deleteSnapshotDiff(bsps, snapshotId, priorSnapshotId, file,
-          collectedBlocks, removedINodes);
+      return diffs.deleteSnapshotDiff(reclaimContext,
+          snapshotId, priorSnapshotId, file);
     }
   }
   
@@ -141,14 +138,12 @@ public class FileWithSnapshotFeature implements INode.Feature {
     this.diffs.clear();
   }
   
-  public QuotaCounts updateQuotaAndCollectBlocks(BlockStoragePolicySuite bsps, INodeFile file,
-      FileDiff removed, BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes) {
-
+  public QuotaCounts updateQuotaAndCollectBlocks(
+      INode.ReclaimContext reclaimContext, INodeFile file, FileDiff removed) {
     byte storagePolicyID = file.getStoragePolicyID();
     BlockStoragePolicy bsp = null;
     if (storagePolicyID != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
-      bsp = bsps.getPolicy(file.getStoragePolicyID());
+      bsp = reclaimContext.storagePolicySuite().getPolicy(file.getStoragePolicyID());
     }
 
 
@@ -180,8 +175,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
       }
     }
 
-    getDiffs().combineAndCollectSnapshotBlocks(
-        bsps, file, removed, collectedBlocks, removedINodes);
+    getDiffs().combineAndCollectSnapshotBlocks(reclaimContext, file, removed);
 
     QuotaCounts current = file.storagespaceConsumed(bsp);
     oldCounts.subtract(current);
@@ -192,11 +186,11 @@ public class FileWithSnapshotFeature implements INode.Feature {
    * If some blocks at the end of the block list no longer belongs to
    * any inode, collect them and update the block list.
    */
-  public void collectBlocksAndClear(final BlockStoragePolicySuite bsps, final INodeFile file,
-      final BlocksMapUpdateInfo info, final List<INode> removedINodes) {
+  public void collectBlocksAndClear(
+      INode.ReclaimContext reclaimContext, final INodeFile file) {
     // check if everything is deleted.
     if (isCurrentFileDeleted() && getDiffs().asList().isEmpty()) {
-      file.destroyAndCollectBlocks(bsps, info, removedINodes, null);
+      file.destroyAndCollectBlocks(reclaimContext);
       return;
     }
     // find max file size.
@@ -212,8 +206,9 @@ public class FileWithSnapshotFeature implements INode.Feature {
     FileDiff last = diffs.getLast();
     BlockInfoContiguous[] snapshotBlocks = last == null ? null : last.getBlocks();
     if(snapshotBlocks == null)
-      file.collectBlocksBeyondMax(max, info);
+      file.collectBlocksBeyondMax(max, reclaimContext.collectedBlocks());
     else
-      file.collectBlocksBeyondSnapshot(snapshotBlocks, info);
+      file.collectBlocksBeyondSnapshot(snapshotBlocks,
+                                       reclaimContext.collectedBlocks());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
index 27d2986..bc9544b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
@@ -235,8 +235,10 @@ public class SnapshotManager implements SnapshotStatsMXBean {
       BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
       throws IOException {
     INodeDirectory srcRoot = getSnapshottableRoot(iip);
-    srcRoot.removeSnapshot(fsdir.getBlockStoragePolicySuite(), snapshotName,
-        collectedBlocks, removedINodes);
+    srcRoot.removeSnapshot(
+        new INode.ReclaimContext(fsdir.getBlockStoragePolicySuite(),
+                                 collectedBlocks, removedINodes, null),
+        snapshotName);
     numSnapshots.getAndDecrement();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51812970/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
index 977b07c..831d65d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
@@ -62,8 +62,9 @@ public class TestFileWithSnapshotFeature {
     INode.BlocksMapUpdateInfo collectedBlocks = mock(
         INode.BlocksMapUpdateInfo.class);
     ArrayList<INode> removedINodes = new ArrayList<>();
-    QuotaCounts counts = sf.updateQuotaAndCollectBlocks(
-        bsps, file, diff, collectedBlocks, removedINodes);
+    INode.ReclaimContext ctx = new INode.ReclaimContext(
+        bsps, collectedBlocks, removedINodes, null);
+    QuotaCounts counts = sf.updateQuotaAndCollectBlocks(ctx, file, diff);
     Assert.assertEquals(0, counts.getStorageSpace());
     Assert.assertTrue(counts.getTypeSpaces().allLessOrEqual(0));
 
@@ -78,8 +79,7 @@ public class TestFileWithSnapshotFeature {
         .thenReturn(Lists.newArrayList(SSD));
     when(bsp.chooseStorageTypes(REPL_3))
         .thenReturn(Lists.newArrayList(DISK));
-    counts = sf.updateQuotaAndCollectBlocks(
-        bsps, file, diff, collectedBlocks, removedINodes);
+    counts = sf.updateQuotaAndCollectBlocks(ctx, file, diff);
     Assert.assertEquals((REPL_3 - REPL_1) * BLOCK_SIZE,
                         counts.getStorageSpace());
     Assert.assertEquals(BLOCK_SIZE, counts.getTypeSpaces().get(DISK));