Explorar o código

HDFS-4563. Update namespace/diskspace usage after deleting snapshots.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1455396 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze %!s(int64=12) %!d(string=hai) anos
pai
achega
3b3ea5c422
Modificáronse 23 ficheiros con 467 adicións e 265 borrados
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
  2. 69 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Content.java
  3. 36 27
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  4. 20 67
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  5. 32 26
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  6. 21 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java
  7. 12 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  8. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  9. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
  10. 7 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Quota.java
  11. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
  12. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
  13. 18 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  14. 24 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshot.java
  15. 13 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
  16. 55 38
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
  17. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java
  18. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java
  19. 5 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java
  20. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/EnumCounters.java
  21. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
  22. 116 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  23. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestDiff.java

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

@@ -192,3 +192,6 @@ Branch-2802 Snapshot (Unreleased)
   HDFS-4579. Annotate snapshot tests. (Arpit Agarwal via suresh)
 
   HDFS-4574. Move Diff to the util package.  (szetszwo)
+
+  HDFS-4563. Update namespace/diskspace usage after deleting snapshots.
+  (Jing Zhao via szetszwo)

+ 69 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Content.java

@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.hdfs.util.EnumCounters;
+
+/**
+ * The content types such as file, directory and symlink to be computed
+ * in {@link INode#computeContentSummary(CountsMap)}.
+ */
+public enum Content {
+  /** The number of files. */
+  FILE,
+  /** The number of directories. */
+  DIRECTORY,
+  /** The number of symlinks. */
+  SYMLINK,
+
+  /** The total of file length in bytes. */
+  LENGTH,
+  /** The total of disk space usage in bytes including replication. */
+  DISKSPACE,
+
+  /** The number of snapshots. */
+  SNAPSHOT,
+  /** The number of snapshottable directories. */
+  SNAPSHOTTABLE_DIRECTORY;
+
+  /** Content counts. */
+  public static class Counts extends EnumCounters<Content> {
+    private Counts() {
+      super(Content.values());
+    }
+  }
+
+  static final EnumCounters.Factory<Content, Counts> FACTORY
+      = new EnumCounters.Factory<Content, Counts>() {
+    @Override
+    public Counts newInstance() {
+      return new Counts();
+    }
+  };
+
+  /** A map of counters for the current state and the snapshots. */
+  public static class CountsMap
+      extends EnumCounters.Map<CountsMap.Key, Content, Counts> {
+    /** The key type of the map. */
+    public static enum Key { CURRENT, SNAPSHOT }
+
+    CountsMap() {
+      super(FACTORY);
+    }
+  }
+}

+ 36 - 27
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -201,7 +201,7 @@ public class FSDirectory implements Closeable {
     ready = flag;
   }
 
-  private void incrDeletedFileCount(int count) {
+  private void incrDeletedFileCount(long count) {
     if (getFSNamesystem() != null)
       NameNode.getNameNodeMetrics().incrFilesDeleted(count);
   }
@@ -757,13 +757,13 @@ public class FSDirectory implements Closeable {
         getFSNamesystem().unprotectedChangeLease(src, dst);
 
         // Collect the blocks and remove the lease for previous dst
-        int filesDeleted = -1;
+        long filesDeleted = -1;
         if (removedDst != null) {
           INode rmdst = removedDst;
           removedDst = null;
           BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
-          filesDeleted = rmdst.cleanSubtree(null,
-              dstIIP.getLatestSnapshot(), collectedBlocks);
+          filesDeleted = rmdst.cleanSubtree(null, dstIIP.getLatestSnapshot(),
+              collectedBlocks).get(Quota.NAMESPACE);
           getFSNamesystem().removePathAndBlocks(src, collectedBlocks);
         }
 
@@ -885,7 +885,7 @@ public class FSDirectory implements Closeable {
   
   void setPermission(String src, FsPermission permission)
       throws FileNotFoundException, UnresolvedLinkException,
-      NSQuotaExceededException, SnapshotAccessControlException {
+      QuotaExceededException, SnapshotAccessControlException {
     writeLock();
     try {
       unprotectedSetPermission(src, permission);
@@ -897,7 +897,7 @@ public class FSDirectory implements Closeable {
   
   void unprotectedSetPermission(String src, FsPermission permissions)
       throws FileNotFoundException, UnresolvedLinkException,
-      NSQuotaExceededException, SnapshotAccessControlException {
+      QuotaExceededException, SnapshotAccessControlException {
     assert hasWriteLock();
     final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(src, true);
     final INode inode = inodesInPath.getLastINode();
@@ -909,7 +909,7 @@ public class FSDirectory implements Closeable {
 
   void setOwner(String src, String username, String groupname)
       throws FileNotFoundException, UnresolvedLinkException,
-      NSQuotaExceededException, SnapshotAccessControlException {
+      QuotaExceededException, SnapshotAccessControlException {
     writeLock();
     try {
       unprotectedSetOwner(src, username, groupname);
@@ -921,7 +921,7 @@ public class FSDirectory implements Closeable {
 
   void unprotectedSetOwner(String src, String username, String groupname)
       throws FileNotFoundException, UnresolvedLinkException,
-      NSQuotaExceededException, SnapshotAccessControlException {
+      QuotaExceededException, SnapshotAccessControlException {
     assert hasWriteLock();
     final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(src, true);
     INode inode = inodesInPath.getLastINode();
@@ -940,7 +940,7 @@ public class FSDirectory implements Closeable {
    * Concat all the blocks from srcs to trg and delete the srcs files
    */
   public void concat(String target, String [] srcs) 
-      throws UnresolvedLinkException, NSQuotaExceededException,
+      throws UnresolvedLinkException, QuotaExceededException,
       SnapshotAccessControlException {
     writeLock();
     try {
@@ -961,7 +961,7 @@ public class FSDirectory implements Closeable {
    * @param srcs list of file to move the blocks from
    */
   void unprotectedConcat(String target, String [] srcs, long timestamp) 
-      throws UnresolvedLinkException, NSQuotaExceededException,
+      throws UnresolvedLinkException, QuotaExceededException,
       SnapshotAccessControlException {
     assert hasWriteLock();
     if (NameNode.stateChangeLog.isDebugEnabled()) {
@@ -1011,7 +1011,7 @@ public class FSDirectory implements Closeable {
     }
     waitForReady();
     long now = now();
-    final int filesRemoved;
+    final long filesRemoved;
     writeLock();
     try {
       final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(
@@ -1100,13 +1100,13 @@ public class FSDirectory implements Closeable {
    * @throws SnapshotAccessControlException if path is in RO snapshot
    */ 
   void unprotectedDelete(String src, long mtime) throws UnresolvedLinkException,
-      NSQuotaExceededException, SnapshotAccessControlException {
+      QuotaExceededException, SnapshotAccessControlException {
     assert hasWriteLock();
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
 
     final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(
         normalizePath(src), false);
-    final int filesRemoved = deleteAllowed(inodesInPath, src)?
+    final long filesRemoved = deleteAllowed(inodesInPath, src)?
         unprotectedDelete(inodesInPath, collectedBlocks, mtime): -1;
     if (filesRemoved >= 0) {
       getFSNamesystem().removePathAndBlocks(src, collectedBlocks);
@@ -1121,8 +1121,8 @@ public class FSDirectory implements Closeable {
    * @param mtime the time the inode is removed
    * @return the number of inodes deleted; 0 if no inodes are deleted.
    */ 
-  int unprotectedDelete(INodesInPath iip, BlocksMapUpdateInfo collectedBlocks,
-      long mtime) throws NSQuotaExceededException {
+  long unprotectedDelete(INodesInPath iip, BlocksMapUpdateInfo collectedBlocks,
+      long mtime) throws QuotaExceededException {
     assert hasWriteLock();
 
     // check if target node exists
@@ -1143,8 +1143,8 @@ public class FSDirectory implements Closeable {
     targetNode.getParent().updateModificationTime(mtime, latestSnapshot);
 
     // collect block
-    final int inodesRemoved = targetNode.cleanSubtree(null, latestSnapshot,
-        collectedBlocks);
+    final long inodesRemoved = targetNode.cleanSubtree(null, latestSnapshot,
+        collectedBlocks).get(Quota.NAMESPACE);
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
           + targetNode.getFullPathName() + " is removed");
@@ -1531,7 +1531,7 @@ public class FSDirectory implements Closeable {
     for(int i=0; i < numOfINodes; i++) {
       if (inodes[i].isQuotaSet()) { // a directory with quota
         INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i]; 
-        node.addSpaceConsumed(nsDelta, dsDelta);
+        node.addSpaceConsumed2Cache(nsDelta, dsDelta);
       }
     }
   }
@@ -1937,7 +1937,7 @@ public class FSDirectory implements Closeable {
    * @throws NSQuotaExceededException 
    */
   private INode removeLastINode(final INodesInPath inodesInPath)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     final Snapshot latestSnapshot = inodesInPath.getLatestSnapshot();
     final INode[] inodes = inodesInPath.getINodes();
     final int pos = inodes.length - 1;
@@ -2020,12 +2020,22 @@ public class FSDirectory implements Closeable {
       }        
 
       final Snapshot latest = iip.getLatestSnapshot();
-      if (dirNode instanceof INodeDirectoryWithQuota) { 
+      if (dirNode instanceof INodeDirectoryWithQuota) {
+        INodeDirectoryWithQuota quotaNode = (INodeDirectoryWithQuota) dirNode;
+        Quota.Counts counts = null;
+        if (!quotaNode.isQuotaSet()) {
+          // dirNode must be an INodeDirectoryWithSnapshot whose quota has not
+          // been set yet
+          counts = quotaNode.computeQuotaUsage();
+        }
         // a directory with quota; so set the quota to the new value
-        ((INodeDirectoryWithQuota)dirNode).setQuota(nsQuota, dsQuota);
-        if (!dirNode.isQuotaSet() && latest == null) {
+        quotaNode.setQuota(nsQuota, dsQuota);
+        if (quotaNode.isQuotaSet() && counts != null) {
+          quotaNode.setSpaceConsumed(counts.get(Quota.NAMESPACE),
+              counts.get(Quota.DISKSPACE));
+        } else if (!quotaNode.isQuotaSet() && latest == null) {
           // will not come here for root because root's nsQuota is always set
-          return dirNode.replaceSelf4INodeDirectory();
+          return quotaNode.replaceSelf4INodeDirectory();
         }
       } else {
         // a non-quota directory; so replace it with a directory with quota
@@ -2069,7 +2079,7 @@ public class FSDirectory implements Closeable {
    * Sets the access time on the file/directory. Logs it in the transaction log.
    */
   void setTimes(String src, INode inode, long mtime, long atime, boolean force,
-      Snapshot latest) throws NSQuotaExceededException {
+      Snapshot latest) throws QuotaExceededException {
     boolean status = false;
     writeLock();
     try {
@@ -2083,7 +2093,7 @@ public class FSDirectory implements Closeable {
   }
 
   boolean unprotectedSetTimes(String src, long mtime, long atime, boolean force) 
-      throws UnresolvedLinkException, NSQuotaExceededException {
+      throws UnresolvedLinkException, QuotaExceededException {
     assert hasWriteLock();
     final INodesInPath i = getLastINodeInPath(src); 
     return unprotectedSetTimes(src, i.getLastINode(), mtime, atime, force,
@@ -2091,8 +2101,7 @@ public class FSDirectory implements Closeable {
   }
 
   private boolean unprotectedSetTimes(String src, INode inode, long mtime,
-      long atime, boolean force, Snapshot latest)
-          throws NSQuotaExceededException {
+      long atime, boolean force, Snapshot latest) throws QuotaExceededException {
     assert hasWriteLock();
     boolean status = false;
     if (mtime != -1) {

+ 20 - 67
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -32,8 +32,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.server.namenode.INode.Content.CountsMap.Key;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.Content.CountsMap.Key;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -43,6 +43,7 @@ import org.apache.hadoop.util.StringUtils;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.SignedBytes;
+//import org.apache.hadoop.hdfs.util.EnumCounters;
 
 /**
  * We keep an in-memory representation of the file/block hierarchy.
@@ -190,7 +191,7 @@ public abstract class INode implements Diff.Element<byte[]> {
   }
   /** Set user */
   final INode setUser(String user, Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setUser(user);
     return nodeToUpdate;
@@ -220,7 +221,7 @@ public abstract class INode implements Diff.Element<byte[]> {
   }
   /** Set group */
   final INode setGroup(String group, Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setGroup(group);
     return nodeToUpdate;
@@ -253,7 +254,7 @@ public abstract class INode implements Diff.Element<byte[]> {
   }
   /** Set the {@link FsPermission} of this {@link INode} */
   INode setPermission(FsPermission permission, Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setPermission(permission);
     return nodeToUpdate;
@@ -286,7 +287,7 @@ public abstract class INode implements Diff.Element<byte[]> {
    *         for maintaining snapshots. The current inode is then the new inode.
    */
   abstract INode recordModification(final Snapshot latest)
-      throws NSQuotaExceededException;
+      throws QuotaExceededException;
 
   /**
    * Check whether it's a file.
@@ -367,10 +368,11 @@ public abstract class INode implements Diff.Element<byte[]> {
    * @param collectedBlocks
    *          blocks collected from the descents for further block
    *          deletion/update will be added to the given map.
-   * @return the number of deleted inodes in the subtree.
+   * @return quota usage delta when deleting a snapshot
    */
-  public abstract int cleanSubtree(final Snapshot snapshot, Snapshot prior,
-      BlocksMapUpdateInfo collectedBlocks) throws NSQuotaExceededException;
+  public abstract Quota.Counts cleanSubtree(final Snapshot snapshot,
+      Snapshot prior, BlocksMapUpdateInfo collectedBlocks)
+      throws QuotaExceededException;
   
   /**
    * Destroy self and clear everything! If the INode is a file, this method
@@ -381,60 +383,10 @@ public abstract class INode implements Diff.Element<byte[]> {
    * 
    * @param collectedBlocks blocks collected from the descents for further block
    *                        deletion/update will be added to this map.
-   * @return the number of deleted inodes in the subtree.
    */
-  public abstract int destroyAndCollectBlocks(
+  public abstract void destroyAndCollectBlocks(
       BlocksMapUpdateInfo collectedBlocks);
 
-  /**
-   * The content types such as file, directory and symlink to be computed
-   * in {@link INode#computeContentSummary(CountsMap)}.
-   */
-  public enum Content {
-    /** The number of files. */
-    FILE,
-    /** The number of directories. */
-    DIRECTORY,
-    /** The number of symlinks. */
-    SYMLINK,
-
-    /** The total of file length in bytes. */
-    LENGTH,
-    /** The total of disk space usage in bytes including replication. */
-    DISKSPACE,
-
-    /** The number of snapshots. */
-    SNAPSHOT,
-    /** The number of snapshottable directories. */
-    SNAPSHOTTABLE_DIRECTORY;
-
-    /** Content counts. */
-    public static class Counts extends EnumCounters<Content> {
-      private Counts() {
-        super(Content.values());
-      }
-    }
-
-    private static final EnumCounters.Factory<Content, Counts> FACTORY
-        = new EnumCounters.Factory<Content, Counts>() {
-      @Override
-      public Counts newInstance() {
-        return new Counts();
-      }
-    };
-
-    /** A map of counters for the current state and the snapshots. */
-    public static class CountsMap
-        extends EnumCounters.Map<CountsMap.Key, Content, Counts> {
-      /** The key type of the map. */
-      public static enum Key { CURRENT, SNAPSHOT }
-
-      private CountsMap() {
-        super(FACTORY);
-      }
-    }
-  }
-
   /** Compute {@link ContentSummary}. */
   public final ContentSummary computeContentSummary() {
     final Content.Counts current = computeContentSummary(
@@ -463,12 +415,13 @@ public abstract class INode implements Diff.Element<byte[]> {
   public abstract Content.Counts computeContentSummary(Content.Counts counts);
   
   /**
-   * Check and add namespace consumed to itself and the ancestors.
-   * @throws NSQuotaExceededException if quote is violated.
+   * Check and add namespace/diskspace consumed to itself and the ancestors.
+   * @throws QuotaExceededException if quote is violated.
    */
-  public void addNamespaceConsumed(int delta) throws NSQuotaExceededException {
+  public void addSpaceConsumed(long nsDelta, long dsDelta)
+      throws QuotaExceededException {
     if (parent != null) {
-      parent.addNamespaceConsumed(delta);
+      parent.addSpaceConsumed(nsDelta, dsDelta);
     }
   }
 
@@ -604,7 +557,7 @@ public abstract class INode implements Diff.Element<byte[]> {
 
   /** Update modification time if it is larger than the current value. */
   public final INode updateModificationTime(long mtime, Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     Preconditions.checkState(isDirectory());
     if (mtime <= modificationTime) {
       return this;
@@ -622,7 +575,7 @@ public abstract class INode implements Diff.Element<byte[]> {
   }
   /** Set the last modification time of inode. */
   public final INode setModificationTime(long modificationTime, Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setModificationTime(modificationTime);
     return nodeToUpdate;
@@ -657,7 +610,7 @@ public abstract class INode implements Diff.Element<byte[]> {
    * Set last access time of inode.
    */
   public INode setAccessTime(long accessTime, Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setAccessTime(accessTime);
     return nodeToUpdate;

+ 32 - 26
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -29,9 +29,9 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
-import org.apache.hadoop.hdfs.server.namenode.INode.Content.CountsMap.Key;
+import org.apache.hadoop.hdfs.server.namenode.Content.CountsMap.Key;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot;
@@ -126,7 +126,7 @@ public class INodeDirectory extends INode {
    * @param latest See {@link INode#recordModification(Snapshot)}.
    */
   public boolean removeChild(INode child, Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     if (isInLatestSnapshot(latest)) {
       return replaceSelf4INodeDirectoryWithSnapshot()
           .removeChild(child, latest);
@@ -167,7 +167,7 @@ public class INodeDirectory extends INode {
    * {@link INodeDirectoryWithSnapshot} depending on the latest snapshot.
    */
   INodeDirectoryWithQuota replaceSelf4Quota(final Snapshot latest,
-      final long nsQuota, final long dsQuota) throws NSQuotaExceededException {
+      final long nsQuota, final long dsQuota) throws QuotaExceededException {
     Preconditions.checkState(!(this instanceof INodeDirectoryWithQuota),
         "this is already an INodeDirectoryWithQuota, this=%s", this);
 
@@ -184,7 +184,7 @@ public class INodeDirectory extends INode {
   }
   /** Replace itself with an {@link INodeDirectorySnapshottable}. */
   public INodeDirectorySnapshottable replaceSelf4INodeDirectorySnapshottable(
-      Snapshot latest) throws NSQuotaExceededException {
+      Snapshot latest) throws QuotaExceededException {
     Preconditions.checkState(!(this instanceof INodeDirectorySnapshottable),
         "this is already an INodeDirectorySnapshottable, this=%s", this);
     final INodeDirectorySnapshottable s = new INodeDirectorySnapshottable(this);
@@ -250,7 +250,7 @@ public class INodeDirectory extends INode {
 
   @Override
   public INodeDirectory recordModification(Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     return isInLatestSnapshot(latest)?
         replaceSelf4INodeDirectoryWithSnapshot().recordModification(latest)
         : this;
@@ -262,7 +262,7 @@ public class INodeDirectory extends INode {
    * @return the child inode, which may be replaced.
    */
   public INode saveChild2Snapshot(final INode child, final Snapshot latest,
-      final INode snapshotCopy) throws NSQuotaExceededException {
+      final INode snapshotCopy) throws QuotaExceededException {
     if (latest == null) {
       return child;
     }
@@ -492,7 +492,7 @@ public class INodeDirectory extends INode {
    *         otherwise, return true;
    */
   public boolean addChild(INode node, final boolean setModTime,
-      final Snapshot latest) throws NSQuotaExceededException {
+      final Snapshot latest) throws QuotaExceededException {
     final int low = searchChildren(node.getLocalNameBytes());
     if (low >= 0) {
       return false;
@@ -601,45 +601,51 @@ public class INodeDirectory extends INode {
    * Call {@link INode#cleanSubtree(SnapshotDeletionInfo, BlocksMapUpdateInfo)}
    * recursively down the subtree.
    */
-  public int cleanSubtreeRecursively(final Snapshot snapshot, Snapshot prior,
-      final BlocksMapUpdateInfo collectedBlocks)
-          throws NSQuotaExceededException {
-    int total = 0;
+  public Quota.Counts cleanSubtreeRecursively(final Snapshot snapshot,
+      Snapshot prior, final BlocksMapUpdateInfo collectedBlocks)
+      throws QuotaExceededException {
+    Quota.Counts counts = Quota.Counts.newInstance();
     // 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.
+    // to its latest previous snapshot. (besides, we also need to consider nodes
+    // created after prior but before snapshot. this will be done in 
+    // INodeDirectoryWithSnapshot#cleanSubtree
     Snapshot s = snapshot != null && prior != null ? prior : snapshot;
     for (INode child : getChildrenList(s)) {
-      total += child.cleanSubtree(snapshot, prior, collectedBlocks);
+      Quota.Counts childCounts = child.cleanSubtree(snapshot, prior,
+          collectedBlocks);
+      counts.add(childCounts);
     }
-    return total;
+    return counts;
   }
 
   @Override
-  public int destroyAndCollectBlocks(
+  public void destroyAndCollectBlocks(
       final BlocksMapUpdateInfo collectedBlocks) {
-    int total = 0;
     for (INode child : getChildrenList(null)) {
-      total += child.destroyAndCollectBlocks(collectedBlocks);
+      child.destroyAndCollectBlocks(collectedBlocks);
     }
     clearReferences();
-    total++;
-    return total;
   }
   
   @Override
-  public int cleanSubtree(final Snapshot snapshot, Snapshot prior,
+  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
       final BlocksMapUpdateInfo collectedBlocks)
-          throws NSQuotaExceededException {
-    int total = 0;
+      throws QuotaExceededException {
     if (prior == null && snapshot == null) {
       // destroy the whole subtree and collect blocks that should be deleted
-      total += destroyAndCollectBlocks(collectedBlocks);
+      destroyAndCollectBlocks(collectedBlocks);
+      return Quota.Counts.newInstance();
     } else {
       // process recursively down the subtree
-      total += cleanSubtreeRecursively(snapshot, prior, collectedBlocks);
+      Quota.Counts counts = cleanSubtreeRecursively(snapshot, prior,
+          collectedBlocks);
+      if (isQuotaSet()) {
+        ((INodeDirectoryWithQuota) this).addSpaceConsumed2Cache(
+            -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
+      }
+      return counts;
     }
-    return total;
   }
   
   /**

+ 21 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java

@@ -23,6 +23,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Directory INode class that has a quota restriction
  */
@@ -94,7 +96,7 @@ public class INodeDirectoryWithQuota extends INodeDirectory {
   @Override
   public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
       boolean useCache) {
-    if (useCache) {
+    if (useCache && isQuotaSet()) {
       // use cache value
       counts.add(Quota.NAMESPACE, namespace);
       counts.add(Quota.DISKSPACE, diskspace);
@@ -138,30 +140,30 @@ public class INodeDirectoryWithQuota extends INodeDirectory {
   }
   
   @Override
-  public final void addNamespaceConsumed(final int delta)
-      throws NSQuotaExceededException {
+  public final void addSpaceConsumed(final long nsDelta, final long dsDelta)
+      throws QuotaExceededException {
     if (isQuotaSet()) { 
       // The following steps are important: 
       // check quotas in this inode and all ancestors before changing counts
       // so that no change is made if there is any quota violation.
 
       // (1) verify quota in this inode  
-      verifyNamespaceQuota(delta);
+      verifyQuota(nsDelta, dsDelta);
       // (2) verify quota and then add count in ancestors 
-      super.addNamespaceConsumed(delta);
+      super.addSpaceConsumed(nsDelta, dsDelta);
       // (3) add count in this inode
-      namespace += delta;
+      addSpaceConsumed2Cache(nsDelta, dsDelta);
     } else {
-      super.addNamespaceConsumed(delta);
+      super.addSpaceConsumed(nsDelta, dsDelta);
     }
   }
-
+  
   /** Update the size of the tree
    * 
    * @param nsDelta the change of the tree size
    * @param dsDelta change to disk space occupied
    */
-  void addSpaceConsumed(long nsDelta, long dsDelta) {
+  protected void addSpaceConsumed2Cache(long nsDelta, long dsDelta) {
     namespace += nsDelta;
     diskspace += dsDelta;
   }
@@ -206,4 +208,14 @@ public class INodeDirectoryWithQuota extends INodeDirectory {
   String quotaString() {
     return ", Quota[" + namespaceString() + ", " + diskspaceString() + "]";
   }
+  
+  @VisibleForTesting
+  public long getNamespace() {
+    return this.namespace;
+  }
+  
+  @VisibleForTesting
+  public long getDiskspace() {
+    return this.diskspace;
+  }
 }

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

@@ -25,12 +25,12 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.namenode.INode.Content.CountsMap.Key;
+import org.apache.hadoop.hdfs.server.namenode.Content.CountsMap.Key;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiffList;
@@ -155,7 +155,7 @@ public class INodeFile extends INode implements BlockCollection {
 
   @Override
   public INodeFile recordModification(final Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     return isInLatestSnapshot(latest)?
         getParent().replaceChild4INodeFileWithSnapshot(this)
             .recordModification(latest)
@@ -179,7 +179,7 @@ public class INodeFile extends INode implements BlockCollection {
    */
   @Override
   final INode setPermission(FsPermission permission, Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     return super.setPermission(permission.applyUMask(UMASK), latest);
   }
 
@@ -211,7 +211,7 @@ public class INodeFile extends INode implements BlockCollection {
 
   /** Set the replication factor of this file. */
   public final INodeFile setFileReplication(short replication, Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     final INodeFile nodeToUpdate = recordModification(latest);
     nodeToUpdate.setFileReplication(replication);
     return nodeToUpdate;
@@ -290,20 +290,18 @@ public class INodeFile extends INode implements BlockCollection {
   }
 
   @Override
-  public int cleanSubtree(final Snapshot snapshot, Snapshot prior,
+  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
       final BlocksMapUpdateInfo collectedBlocks)
-          throws NSQuotaExceededException {
+      throws QuotaExceededException {
     if (snapshot == null && prior == null) {   
       // this only happens when deleting the current file
-      return destroyAndCollectBlocks(collectedBlocks);
-    } else {
-      return 0;
+      destroyAndCollectBlocks(collectedBlocks);
     }
+    return Quota.Counts.newInstance();
   }
 
   @Override
-  public int destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks) {
-    int total = 1;
+  public void destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks) {
     if (blocks != null && collectedBlocks != null) {
       for (BlockInfo blk : blocks) {
         collectedBlocks.addDeleteBlock(blk);
@@ -314,9 +312,8 @@ public class INodeFile extends INode implements BlockCollection {
     clearReferences();
     
     if (this instanceof FileWithSnapshot) {
-      total += ((FileWithSnapshot) this).getDiffs().clear();
+      ((FileWithSnapshot) this).getDiffs().clear();
     }
-    return total;
   }
   
   @Override
@@ -442,7 +439,7 @@ public class INodeFile extends INode implements BlockCollection {
     return size;
   }
 
-  final long diskspaceConsumed() {
+  public final long diskspaceConsumed() {
     // use preferred block size for the last block if it is under construction
     return computeFileSize(true, true) * getBlockReplication();
   }

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

@@ -24,7 +24,7 @@ import java.util.Arrays;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -131,7 +131,7 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
   
   @Override
   public INodeFileUnderConstruction recordModification(final Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     return isInLatestSnapshot(latest)?
         getParent().replaceChild4INodeFileUcWithSnapshot(this)
             .recordModification(latest)

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

@@ -22,8 +22,8 @@ import java.io.PrintWriter;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.server.namenode.INode.Content.CountsMap.Key;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.Content.CountsMap.Key;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 /**
@@ -45,7 +45,7 @@ public class INodeSymlink extends INode {
   }
 
   @Override
-  INode recordModification(Snapshot latest) throws NSQuotaExceededException {
+  INode recordModification(Snapshot latest) throws QuotaExceededException {
     return isInLatestSnapshot(latest)?
         getParent().saveChild2Snapshot(this, latest, new INodeSymlink(this))
         : this;
@@ -72,15 +72,15 @@ public class INodeSymlink extends INode {
   }
   
   @Override
-  public int cleanSubtree(final Snapshot snapshot, Snapshot prior,
+  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
       final BlocksMapUpdateInfo collectedBlocks) {
-    return 1;
+    return Quota.Counts.newInstance();
   }
   
   @Override
-  public int destroyAndCollectBlocks(
+  public void destroyAndCollectBlocks(
       final BlocksMapUpdateInfo collectedBlocks) {
-    return 1;
+    // do nothing
   }
 
   @Override

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Quota.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.hadoop.hdfs.util.EnumCounters;
+
 /** Quota types. */
 public enum Quota {
   /** The namespace usage, i.e. the number of name objects. */
@@ -27,13 +29,17 @@ public enum Quota {
   /** Counters for quota counts. */
   public static class Counts extends EnumCounters<Quota> {
     /** @return a new counter with the given namespace and diskspace usages. */
-    static Counts newInstance(long namespace, long diskspace) {
+    public static Counts newInstance(long namespace, long diskspace) {
       final Counts c = new Counts();
       c.set(NAMESPACE, namespace);
       c.set(DISKSPACE, diskspace);
       return c;
     }
 
+    public static Counts newInstance() {
+      return newInstance(0, 0);
+    }
+    
     Counts() {
       super(Quota.values());
     }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java

@@ -145,7 +145,7 @@ public class NameNodeMetrics {
     filesRenamed.incr();
   }
 
-  public void incrFilesDeleted(int delta) {
+  public void incrFilesDeleted(long delta) {
     filesDeleted.incr(delta);
   }
 

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

@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 
 import com.google.common.base.Preconditions;
@@ -126,15 +127,15 @@ abstract class AbstractINodeDiff<N extends INode,
   }
 
   /** Combine the posterior diff and collect blocks for deletion. */
-  abstract int combinePosteriorAndCollectBlocks(final N currentINode,
+  abstract Quota.Counts combinePosteriorAndCollectBlocks(final N currentINode,
       final D posterior, final BlocksMapUpdateInfo collectedBlocks);
   
   /**
    * Delete and clear self.
    * @param collectedBlocks Used to collect blocks for deletion.
-   * @return number of inodes/diff destroyed.
+   * @return quota usage delta
    */
-  abstract int destroyAndCollectBlocks(final N currentINode,
+  abstract Quota.Counts destroyDiffAndCollectBlocks(final N currentINode,
       final BlocksMapUpdateInfo collectedBlocks);
 
   @Override

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

@@ -23,8 +23,10 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 /**
  * A list of snapshot diffs for storing snapshot data.
@@ -50,10 +52,8 @@ abstract class AbstractINodeDiffList<N extends INode,
   }
   
   /** Get the size of the list and then clear it. */
-  public int clear() {
-    final int n = diffs.size();
+  public void clear() {
     diffs.clear();
-    return n;
   }
 
   /**
@@ -66,11 +66,12 @@ abstract class AbstractINodeDiffList<N extends INode,
    * @param collectedBlocks Used to collect information for blocksMap update
    * @return delta in namespace. 
    */
-  final int deleteSnapshotDiff(final Snapshot snapshot, Snapshot prior,
-      final N currentINode, final BlocksMapUpdateInfo collectedBlocks) {
+  final Quota.Counts deleteSnapshotDiff(final Snapshot snapshot,
+      Snapshot prior, final N currentINode,
+      final BlocksMapUpdateInfo collectedBlocks) {
     int snapshotIndex = Collections.binarySearch(diffs, snapshot);
     
-    int removedNum = 0;
+    Quota.Counts counts = Quota.Counts.newInstance();
     D removed = null;
     if (snapshotIndex == 0) {
       if (prior != null) {
@@ -78,9 +79,9 @@ abstract class AbstractINodeDiffList<N extends INode,
         diffs.get(snapshotIndex).setSnapshot(prior);
       } else {
         removed = diffs.remove(0);
-        removedNum++; // removed a diff
-        removedNum += removed.destroyAndCollectBlocks(currentINode,
-            collectedBlocks);
+        counts.add(Quota.NAMESPACE, 1);
+        counts.add(removed.destroyDiffAndCollectBlocks(currentINode,
+            collectedBlocks));
       }
     } else if (snapshotIndex > 0) {
       final AbstractINodeDiff<N, D> previous = diffs.get(snapshotIndex - 1);
@@ -89,25 +90,25 @@ abstract class AbstractINodeDiffList<N extends INode,
       } else {
         // combine the to-be-removed diff with its previous diff
         removed = diffs.remove(snapshotIndex);
-        removedNum++;
+        counts.add(Quota.NAMESPACE, 1);
         if (previous.snapshotINode == null) {
           previous.snapshotINode = removed.snapshotINode;
         } else if (removed.snapshotINode != null) {
           removed.snapshotINode.clearReferences();
         }
-        removedNum += previous.combinePosteriorAndCollectBlocks(currentINode,
-            removed, collectedBlocks);
+        counts.add(previous.combinePosteriorAndCollectBlocks(
+            currentINode, removed, collectedBlocks));
         previous.setPosterior(removed.getPosterior());
         removed.setPosterior(null);
       }
     }
-    return removedNum;
+    return counts;
   }
 
   /** Add an {@link AbstractINodeDiff} for the given snapshot. */
   final D addDiff(Snapshot latest, N currentINode)
-      throws NSQuotaExceededException {
-    currentINode.addNamespaceConsumed(1);
+      throws QuotaExceededException {
+    currentINode.addSpaceConsumed(1, 0);
     return addLast(factory.createDiff(latest, currentINode));
   }
 
@@ -226,7 +227,7 @@ abstract class AbstractINodeDiffList<N extends INode,
    * @return the latest snapshot diff, which is never null.
    */
   final D checkAndAddLatestSnapshotDiff(Snapshot latest, N currentINode)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     final D last = getLast();
     if (last != null
         && Snapshot.ID_COMPARATOR.compare(last.getSnapshot(), latest) >= 0) {
@@ -243,7 +244,7 @@ abstract class AbstractINodeDiffList<N extends INode,
 
   /** Save the snapshot copy to the latest snapshot. */
   public void saveSelf2Snapshot(Snapshot latest, N currentINode, N snapshotCopy)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     if (latest != null) {
       checkAndAddLatestSnapshotDiff(latest, currentINode).saveSnapshotCopy(
           snapshotCopy, factory, currentINode);

+ 24 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshot.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 /**
  * {@link INodeFile} with a link to the next element.
@@ -57,12 +58,30 @@ public interface FileWithSnapshot {
       return fileSize;
     }
 
+    private static Quota.Counts updateQuotaAndCollectBlocks(
+        INodeFile currentINode, FileDiff removed,
+        BlocksMapUpdateInfo collectedBlocks) {
+      FileWithSnapshot sFile = (FileWithSnapshot) currentINode;
+      long oldDiskspace = currentINode.diskspaceConsumed();
+      if (removed.snapshotINode != null) {
+        short replication = removed.snapshotINode.getFileReplication();
+        if (replication > currentINode.getBlockReplication()) {
+          oldDiskspace = oldDiskspace / currentINode.getBlockReplication()
+              * replication;
+        }
+      }
+      
+      Util.collectBlocksAndClear(sFile, collectedBlocks);
+      
+      long dsDelta = oldDiskspace - currentINode.diskspaceConsumed();
+      return Quota.Counts.newInstance(0, dsDelta);
+    }
+    
     @Override
-    int combinePosteriorAndCollectBlocks(INodeFile currentINode,
+    Quota.Counts combinePosteriorAndCollectBlocks(INodeFile currentINode,
         FileDiff posterior, BlocksMapUpdateInfo collectedBlocks) {
-      Util.collectBlocksAndClear((FileWithSnapshot) currentINode,
+      return updateQuotaAndCollectBlocks(currentINode, posterior,
           collectedBlocks);
-      return 0;
     }
     
     @Override
@@ -86,11 +105,10 @@ public interface FileWithSnapshot {
     }
 
     @Override
-    int destroyAndCollectBlocks(INodeFile currentINode,
+    Quota.Counts destroyDiffAndCollectBlocks(INodeFile currentINode,
         BlocksMapUpdateInfo collectedBlocks) {
-      Util.collectBlocksAndClear((FileWithSnapshot) currentINode,
+      return updateQuotaAndCollectBlocks(currentINode, this,
           collectedBlocks);
-      return 0;
     }
   }
 

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

@@ -30,14 +30,16 @@ import java.util.Map;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+import org.apache.hadoop.hdfs.server.namenode.Content;
+import org.apache.hadoop.hdfs.server.namenode.Content.CountsMap.Key;
 import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INode.Content.CountsMap.Key;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.Time;
 
@@ -273,7 +275,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
 
   /** Add a snapshot. */
   Snapshot addSnapshot(int id, String name)
-      throws SnapshotException, NSQuotaExceededException {
+      throws SnapshotException, QuotaExceededException {
     //check snapshot quota
     final int n = getNumSnapshots();
     if (n + 1 > snapshotQuota) {
@@ -319,8 +321,13 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
       final Snapshot snapshot = snapshotsByNames.remove(i);
       Snapshot prior = Snapshot.findLatestSnapshot(this, snapshot);
       try {
-        cleanSubtree(snapshot, prior, collectedBlocks);
-      } catch(NSQuotaExceededException e) {
+        Quota.Counts counts = cleanSubtree(snapshot, prior, collectedBlocks);
+        INodeDirectory parent = getParent();
+        if (parent != null) {
+          parent.addSpaceConsumed(counts.get(Quota.NAMESPACE),
+              counts.get(Quota.DISKSPACE));
+        }
+      } catch(QuotaExceededException e) {
         LOG.error("BUG: removeSnapshot increases namespace usage.", e);
       }
       return snapshot;
@@ -434,7 +441,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
    * Replace itself with {@link INodeDirectoryWithSnapshot} or
    * {@link INodeDirectory} depending on the latest snapshot.
    */
-  void replaceSelf(final Snapshot latest) throws NSQuotaExceededException {
+  void replaceSelf(final Snapshot latest) throws QuotaExceededException {
     if (latest == null) {
       Preconditions.checkState(getLastSnapshot() == null,
           "latest == null but getLastSnapshot() != null, this=%s", this);

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

@@ -25,12 +25,13 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+import org.apache.hadoop.hdfs.server.namenode.Content;
+import org.apache.hadoop.hdfs.server.namenode.Content.CountsMap.Key;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INode.Content.CountsMap.Key;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
@@ -73,29 +74,29 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
 
     /** clear the created list */
-    private int destroyCreatedList(
+    private void destroyCreatedList(
         final INodeDirectoryWithSnapshot currentINode,
         final BlocksMapUpdateInfo collectedBlocks) {
-      int removedNum = 0;
       List<INode> createdList = getCreatedList();
       for (INode c : createdList) {
-        removedNum += c.destroyAndCollectBlocks(collectedBlocks);
-        // if c is also contained in the children list, remove it
+        c.destroyAndCollectBlocks(collectedBlocks);
+        // c should be contained in the children list, remove it
         currentINode.removeChild(c);
       }
       createdList.clear();
-      return removedNum;
     }
     
     /** clear the deleted list */
-    private int destroyDeletedList(final BlocksMapUpdateInfo collectedBlocks) {
-      int removedNum  = 0;
+    private Quota.Counts destroyDeletedList(
+        final BlocksMapUpdateInfo collectedBlocks) {
+      Quota.Counts counts = Quota.Counts.newInstance();
       List<INode> deletedList = getDeletedList();
       for (INode d : deletedList) {
-        removedNum += d.destroyAndCollectBlocks(collectedBlocks);
+        d.computeQuotaUsage(counts, false);
+        d.destroyAndCollectBlocks(collectedBlocks);
       }
       deletedList.clear();
-      return removedNum;
+      return counts;
     }
     
     /** Serialize {@link #created} */
@@ -233,18 +234,21 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
     
     @Override
-    int combinePosteriorAndCollectBlocks(final INodeDirectory currentDir,
-        final DirectoryDiff posterior, final BlocksMapUpdateInfo collectedBlocks) {
-      return diff.combinePosterior(posterior.diff, new Diff.Processor<INode>() {
+    Quota.Counts combinePosteriorAndCollectBlocks(
+        final INodeDirectory currentDir, final DirectoryDiff posterior,
+        final BlocksMapUpdateInfo collectedBlocks) {
+      final Quota.Counts counts = Quota.Counts.newInstance();
+      diff.combinePosterior(posterior.diff, new Diff.Processor<INode>() {
         /** Collect blocks for deleted files. */
         @Override
-        public int process(INode inode) {
+        public void process(INode inode) {
           if (inode != null) {
-            return inode.destroyAndCollectBlocks(collectedBlocks);
+            inode.computeQuotaUsage(counts, false);
+            inode.destroyAndCollectBlocks(collectedBlocks);
           }
-          return 0;
         }
       });
+      return counts;
     }
 
     /**
@@ -334,9 +338,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
 
     @Override
-    int destroyAndCollectBlocks(INodeDirectory currentINode,
+    Quota.Counts destroyDiffAndCollectBlocks(INodeDirectory currentINode,
         BlocksMapUpdateInfo collectedBlocks) {
-      return diff.destroyDeletedList(collectedBlocks);      
+      // this diff has been deleted
+      Quota.Counts counts = Quota.Counts.newInstance();
+      counts.add(diff.destroyDeletedList(collectedBlocks));
+      return counts;
     }
   }
 
@@ -465,7 +472,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
 
   @Override
   public INodeDirectoryWithSnapshot recordModification(final Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     return isInLatestSnapshot(latest)?
         saveSelf2Snapshot(latest, null): this;
   }
@@ -473,14 +480,14 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   /** Save the snapshot copy to the latest snapshot. */
   public INodeDirectoryWithSnapshot saveSelf2Snapshot(
       final Snapshot latest, final INodeDirectory snapshotCopy)
-          throws NSQuotaExceededException {
+          throws QuotaExceededException {
     diffs.saveSelf2Snapshot(latest, this, snapshotCopy);
     return this;
   }
 
   @Override
   public INode saveChild2Snapshot(final INode child, final Snapshot latest,
-      final INode snapshotCopy) throws NSQuotaExceededException {
+      final INode snapshotCopy) throws QuotaExceededException {
     Preconditions.checkArgument(!child.isDirectory(),
         "child is a directory, child=%s", child);
     if (latest == null) {
@@ -499,7 +506,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
 
   @Override
   public boolean addChild(INode inode, boolean setModTime, Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     ChildrenDiff diff = null;
     Integer undoInfo = null;
     if (latest != null) {
@@ -515,7 +522,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
 
   @Override
   public boolean removeChild(INode child, Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     ChildrenDiff diff = null;
     UndoInfo<INode> undoInfo = null;
     if (latest != null) {
@@ -610,16 +617,16 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   }
 
   @Override
-  public int cleanSubtree(final Snapshot snapshot, Snapshot prior,
+  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
       final BlocksMapUpdateInfo collectedBlocks)
-          throws NSQuotaExceededException {
-    int n = 0;
+      throws QuotaExceededException {
+    Quota.Counts counts = Quota.Counts.newInstance();
     if (snapshot == null) { // delete the current directory
       recordModification(prior);
       // delete everything in created list
       DirectoryDiff lastDiff = diffs.getLast();
       if (lastDiff != null) {
-        n += lastDiff.diff.destroyCreatedList(this, collectedBlocks);
+        lastDiff.diff.destroyCreatedList(this, collectedBlocks);
       }
     } else {
       // update prior
@@ -628,25 +635,35 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
           (prior == null || Snapshot.ID_COMPARATOR.compare(s, prior) > 0)) {
         prior = s;
       }
-      n += getDiffs().deleteSnapshotDiff(snapshot, prior, this, 
-          collectedBlocks);
+      counts.add(getDiffs().deleteSnapshotDiff(snapshot, prior, this, 
+          collectedBlocks));
+      if (prior != null) {
+        DirectoryDiff priorDiff = this.getDiffs().getDiff(prior);
+        if (priorDiff != null) {
+          for (INode cNode : priorDiff.getChildrenDiff().getCreatedList()) {
+            counts.add(cNode.cleanSubtree(snapshot, null, collectedBlocks));
+          }
+        }
+      }
     }
+    counts.add(cleanSubtreeRecursively(snapshot, prior, collectedBlocks));
     
-    n += cleanSubtreeRecursively(snapshot, prior, collectedBlocks);
-    return n;
+    if (isQuotaSet()) {
+      this.addSpaceConsumed2Cache(-counts.get(Quota.NAMESPACE),
+          -counts.get(Quota.DISKSPACE));
+    }
+    return counts;
   }
 
   @Override
-  public int destroyAndCollectBlocks(
+  public void destroyAndCollectBlocks(
       final BlocksMapUpdateInfo collectedBlocks) {
-    int total = 0;
     // destroy its diff list
     for (DirectoryDiff diff : diffs) {
-      total += diff.destroyAndCollectBlocks(this, collectedBlocks);
+      diff.destroyDiffAndCollectBlocks(this, collectedBlocks);
     }
-    total += diffs.clear();
-    total += super.destroyAndCollectBlocks(collectedBlocks);
-    return total;
+    diffs.clear();
+    super.destroyAndCollectBlocks(collectedBlocks);
   }
 
   @Override

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

@@ -18,10 +18,11 @@
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 /**
  * Represent an {@link INodeFileUnderConstruction} that is snapshotted.
@@ -93,7 +94,7 @@ public class INodeFileUnderConstructionWithSnapshot
 
   @Override
   public INodeFileUnderConstructionWithSnapshot recordModification(
-      final Snapshot latest) throws NSQuotaExceededException {
+      final Snapshot latest) throws QuotaExceededException {
     if (isInLatestSnapshot(latest)) {
       diffs.saveSelf2Snapshot(latest, this, null);
     }
@@ -111,17 +112,17 @@ public class INodeFileUnderConstructionWithSnapshot
   }
 
   @Override
-  public int cleanSubtree(final Snapshot snapshot, Snapshot prior,
+  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
       final BlocksMapUpdateInfo collectedBlocks)
-          throws NSQuotaExceededException {
+      throws QuotaExceededException {
     if (snapshot == null) { // delete the current file
       recordModification(prior);
       isCurrentFileDeleted = true;
       Util.collectBlocksAndClear(this, collectedBlocks);
+      return Quota.Counts.newInstance();
     } else { // delete a snapshot
       return diffs.deleteSnapshotDiff(snapshot, prior, this, collectedBlocks);
     }
-    return prior == null ? 1 : 0;
   }
 
   @Override

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

@@ -18,9 +18,10 @@
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 /**
  * Represent an {@link INodeFile} that is snapshotted.
@@ -64,7 +65,7 @@ public class INodeFileWithSnapshot extends INodeFile
 
   @Override
   public INodeFileWithSnapshot recordModification(final Snapshot latest)
-      throws NSQuotaExceededException {
+      throws QuotaExceededException {
     if (isInLatestSnapshot(latest)) {
       diffs.saveSelf2Snapshot(latest, this, null);
     }
@@ -82,17 +83,17 @@ public class INodeFileWithSnapshot extends INodeFile
   }
 
   @Override
-  public int cleanSubtree(final Snapshot snapshot, Snapshot prior, 
+  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
       final BlocksMapUpdateInfo collectedBlocks)
-          throws NSQuotaExceededException {
+      throws QuotaExceededException {
     if (snapshot == null) { // delete the current file
       recordModification(prior);
       isCurrentFileDeleted = true;
       Util.collectBlocksAndClear(this, collectedBlocks);
+      return Quota.Counts.newInstance();
     } else { // delete a snapshot
       return diffs.deleteSnapshotDiff(snapshot, prior, this, collectedBlocks);
     }
-    return prior == null ? 1 : 0;
   }
 
   @Override

+ 5 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java

@@ -82,7 +82,7 @@ public class Diff<K, E extends Diff.Element<K>> {
   /** An interface for passing a method in order to process elements. */
   public static interface Processor<E> {
     /** Process the given element. */
-    public int process(E element);
+    public void process(E element);
   }
 
   /** Containing exactly one element. */
@@ -153,7 +153,7 @@ public class Diff<K, E extends Diff.Element<K>> {
   }
 
   /** @return the created list, which is never null. */
-  protected List<E> getCreatedList() {
+  public List<E> getCreatedList() {
     return created == null? Collections.<E>emptyList(): created;
   }
 
@@ -420,7 +420,7 @@ public class Diff<K, E extends Diff.Element<K>> {
    * @param deletedProcesser
    *     process the deleted/overwritten elements in case 2.1, 2.3, 3.1 and 3.3.
    */
-  public int combinePosterior(final Diff<K, E> posterior,
+  public void combinePosterior(final Diff<K, E> posterior,
       final Processor<E> deletedProcesser) {
     final Iterator<E> createdIterator = posterior.getCreatedList().iterator();
     final Iterator<E> deletedIterator = posterior.getDeletedList().iterator();
@@ -428,7 +428,6 @@ public class Diff<K, E extends Diff.Element<K>> {
     E c = createdIterator.hasNext()? createdIterator.next(): null;
     E d = deletedIterator.hasNext()? deletedIterator.next(): null;
 
-    int deletedNum = 0;
     for(; c != null || d != null; ) {
       final int cmp = c == null? 1
           : d == null? -1
@@ -441,20 +440,19 @@ public class Diff<K, E extends Diff.Element<K>> {
         // case 2: only in d-list
         final UndoInfo<E> ui = delete(d);
         if (deletedProcesser != null) {
-          deletedNum += deletedProcesser.process(ui.trashed);
+          deletedProcesser.process(ui.trashed);
         }
         d = deletedIterator.hasNext()? deletedIterator.next(): null;
       } else {
         // case 3: in both c-list and d-list 
         final UndoInfo<E> ui = modify(d, c);
         if (deletedProcesser != null) {
-          deletedNum += deletedProcesser.process(ui.trashed);
+          deletedProcesser.process(ui.trashed);
         }
         c = createdIterator.hasNext()? createdIterator.next(): null;
         d = deletedIterator.hasNext()? deletedIterator.next(): null;
       }
     }
-    return deletedNum;
   }
 
   @Override

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EnumCounters.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/EnumCounters.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.util;
 
 import java.util.HashMap;
 

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java

@@ -173,9 +173,7 @@ public class TestFSImageWithSnapshot {
     int s = 0;
     // make changes to the namesystem
     hdfs.mkdirs(dir);
-    hdfs.allowSnapshot(dir.toString());
-
-    hdfs.createSnapshot(dir, "s" + ++s);
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s" + ++s);
     Path sub1 = new Path(dir, "sub1");
     hdfs.mkdirs(sub1);
     hdfs.setPermission(sub1, new FsPermission((short)0777));

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

@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.FileNotFoundException;
+import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -37,6 +38,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -141,6 +143,17 @@ public class TestSnapshotDeletion {
     hdfs.delete(dir, true);
   }
   
+  private void checkQuotaUsageComputation(final Path dirPath,
+      final long expectedNs, final long expectedDs) throws IOException {
+    INode node = fsdir.getINode(dirPath.toString());
+    assertTrue(node.isDirectory() && node.isQuotaSet());
+    INodeDirectoryWithQuota dirNode = (INodeDirectoryWithQuota) node;
+    assertEquals(dirNode.dumpTreeRecursively().toString(), expectedNs,
+        dirNode.getNamespace());
+    assertEquals(dirNode.dumpTreeRecursively().toString(), expectedDs,
+        dirNode.getDiskspace());
+  }
+  
   /**
    * Test deleting a directory which is a descendant of a snapshottable
    * directory. In the test we need to cover the following cases:
@@ -177,14 +190,20 @@ public class TestSnapshotDeletion {
     
     // create snapshot s0
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
+    // check dir's quota usage
+    checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 3);
+    
     // make a change: create a new file under subsub
     Path newFileAfterS0 = new Path(subsub, "newFile");
     DFSTestUtil.createFile(hdfs, newFileAfterS0, BLOCKSIZE, REPLICATION, seed);
     // further change: change the replicator factor of metaChangeFile
     hdfs.setReplication(metaChangeFile1, REPLICATION_1);
     hdfs.setReplication(metaChangeFile2, REPLICATION_1);
+    
     // create snapshot s1
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
+    // check dir's quota usage
+    checkQuotaUsageComputation(dir, 14L, BLOCKSIZE * REPLICATION * 4);
     
     // get two snapshots for later use
     Snapshot snapshot0 = ((INodeDirectorySnapshottable) fsdir.getINode(dir
@@ -198,6 +217,9 @@ public class TestSnapshotDeletion {
     // sure the deletion goes through an INodeDirectory, we delete the parent
     // of noChangeDir
     hdfs.delete(noChangeDirParent, true);
+    // while deletion, we add a diff for metaChangeFile2 as its snapshot copy
+    // for s1, we also add diffs for both sub and noChangeDirParent
+    checkQuotaUsageComputation(dir, 17L, BLOCKSIZE * REPLICATION * 4);
     
     // check the snapshot copy of noChangeDir 
     Path snapshotNoChangeDir = SnapshotTestHelper.getSnapshotPath(dir, "s1",
@@ -208,7 +230,7 @@ public class TestSnapshotDeletion {
     // should still be an INodeDirectory
     assertEquals(INodeDirectory.class, snapshotNode.getClass());
     ReadOnlyList<INode> children = snapshotNode.getChildrenList(null);
-    // check 2 children: noChangeFile and metaChangeFile1
+    // check 2 children: noChangeFile and metaChangeFile2
     assertEquals(2, children.size());
     INode noChangeFileSCopy = children.get(1);
     assertEquals(noChangeFile.getName(), noChangeFileSCopy.getLocalName());
@@ -229,7 +251,10 @@ public class TestSnapshotDeletion {
     // before deleting sub, we first create a new file under sub
     Path newFile = new Path(sub, "newFile");
     DFSTestUtil.createFile(hdfs, newFile, BLOCKSIZE, REPLICATION, seed);
+    checkQuotaUsageComputation(dir, 18L, BLOCKSIZE * REPLICATION * 5);
     hdfs.delete(sub, true);
+    // while deletion, we add diff for dir and metaChangeFile1
+    checkQuotaUsageComputation(dir, 20L, BLOCKSIZE * REPLICATION * 5);
     
     // make sure the whole subtree of sub is stored correctly in snapshot
     Path snapshotSub = SnapshotTestHelper.getSnapshotPath(dir, "s1",
@@ -307,11 +332,15 @@ public class TestSnapshotDeletion {
     }
     
     // create snapshot s1 for sub
-    hdfs.createSnapshot(sub, snapshotName);
+    SnapshotTestHelper.createSnapshot(hdfs, sub, snapshotName);
+    // check quota usage computation
+    checkQuotaUsageComputation(sub, 4, BLOCKSIZE * REPLICATION * 2);
     // delete s1
     hdfs.deleteSnapshot(sub, snapshotName);
+    checkQuotaUsageComputation(sub, 3, BLOCKSIZE * REPLICATION * 2);
     // now we can create a snapshot with the same name
     hdfs.createSnapshot(sub, snapshotName);
+    checkQuotaUsageComputation(sub, 4, BLOCKSIZE * REPLICATION * 2);
     
     // create a new file under sub
     Path newFile = new Path(sub, "newFile");
@@ -319,12 +348,14 @@ public class TestSnapshotDeletion {
     // create another snapshot s2
     String snapshotName2 = "s2";
     hdfs.createSnapshot(sub, snapshotName2);
+    checkQuotaUsageComputation(sub, 6, BLOCKSIZE * REPLICATION * 3);
     // Get the filestatus of sub under snapshot s2
     Path ss = SnapshotTestHelper
         .getSnapshotPath(sub, snapshotName2, "newFile");
     FileStatus statusBeforeDeletion = hdfs.getFileStatus(ss);
     // delete s1
     hdfs.deleteSnapshot(sub, snapshotName);
+    checkQuotaUsageComputation(sub, 5, BLOCKSIZE * REPLICATION * 3);
     FileStatus statusAfterDeletion = hdfs.getFileStatus(ss);
     System.out.println("Before deletion: " + statusBeforeDeletion.toString()
         + "\n" + "After deletion: " + statusAfterDeletion.toString());
@@ -356,19 +387,28 @@ public class TestSnapshotDeletion {
     
     // create snapshot s0 on dir
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
+    checkQuotaUsageComputation(dir, 8, 3 * BLOCKSIZE * REPLICATION);
     
     // delete /TestSnapshot/sub/noChangeDir/metaChangeDir/toDeleteFile
     hdfs.delete(toDeleteFile, true);
+    // the deletion adds diff of toDeleteFile and metaChangeDir
+    checkQuotaUsageComputation(dir, 10, 3 * BLOCKSIZE * REPLICATION);
     // change metadata of /TestSnapshot/sub/noChangeDir/metaChangeDir and
     // /TestSnapshot/sub/noChangeDir/metaChangeFile
     hdfs.setReplication(metaChangeFile, REPLICATION_1);
     hdfs.setOwner(metaChangeDir, "unknown", "unknown");
+    checkQuotaUsageComputation(dir, 11, 3 * BLOCKSIZE * REPLICATION);
     
     // create snapshot s1 on dir
     hdfs.createSnapshot(dir, "s1");
+    checkQuotaUsageComputation(dir, 12, 3 * BLOCKSIZE * REPLICATION);
     
     // delete snapshot s0
     hdfs.deleteSnapshot(dir, "s0");
+    // namespace: remove toDeleteFile and its diff, metaChangeFile's diff, 
+    // metaChangeDir's diff, dir's diff. diskspace: remove toDeleteFile, and 
+    // metaChangeFile's replication factor decreases
+    checkQuotaUsageComputation(dir, 7, 2 * BLOCKSIZE * REPLICATION - BLOCKSIZE);
     
     // check 1. there is no snapshot s0
     final INodeDirectorySnapshottable dirNode = 
@@ -423,7 +463,7 @@ public class TestSnapshotDeletion {
    */
   @Test (timeout=300000)
   public void testCombineSnapshotDiff1() throws Exception {
-    testCombineSnapshotDiffImpl(sub, "");
+    testCombineSnapshotDiffImpl(sub, "", 1);
   }
   
   /**
@@ -432,7 +472,7 @@ public class TestSnapshotDeletion {
    */
   @Test (timeout=300000)
   public void testCombineSnapshotDiff2() throws Exception {
-    testCombineSnapshotDiffImpl(sub, "subsub1/subsubsub1/");
+    testCombineSnapshotDiffImpl(sub, "subsub1/subsubsub1/", 3);
   }
   
   /**
@@ -442,54 +482,90 @@ public class TestSnapshotDeletion {
    *        where the modifications happen. It is represented as a relative 
    *        path to the snapshotRoot.
    */
-  private void testCombineSnapshotDiffImpl(Path snapshotRoot, String modDirStr)
-      throws Exception {
+  private void testCombineSnapshotDiffImpl(Path snapshotRoot, String modDirStr,
+      int dirNodeNum) throws Exception {
     Path modDir = modDirStr.isEmpty() ? snapshotRoot : new Path(snapshotRoot,
         modDirStr);
+    final int delta = modDirStr.isEmpty() ? 0 : 1;
     Path file10 = new Path(modDir, "file10");
     Path file11 = new Path(modDir, "file11");
     Path file12 = new Path(modDir, "file12");
     Path file13 = new Path(modDir, "file13");
     Path file14 = new Path(modDir, "file14");
     Path file15 = new Path(modDir, "file15");
-    final short REP_1 = REPLICATION - 1;
-    DFSTestUtil.createFile(hdfs, file10, BLOCKSIZE, REP_1, seed);
-    DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, REP_1, seed);
-    DFSTestUtil.createFile(hdfs, file12, BLOCKSIZE, REP_1, seed);
-    DFSTestUtil.createFile(hdfs, file13, BLOCKSIZE, REP_1, seed);
+    DFSTestUtil.createFile(hdfs, file10, BLOCKSIZE, REPLICATION_1, seed);
+    DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, REPLICATION_1, seed);
+    DFSTestUtil.createFile(hdfs, file12, BLOCKSIZE, REPLICATION_1, seed);
+    DFSTestUtil.createFile(hdfs, file13, BLOCKSIZE, REPLICATION_1, seed);
 
     // create snapshot s1 for snapshotRoot
     SnapshotTestHelper.createSnapshot(hdfs, snapshotRoot, "s1");
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 5, 8 * BLOCKSIZE);
     
     // delete file11
     hdfs.delete(file11, true);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 6 + delta,
+        8 * BLOCKSIZE);
+    
     // modify file12
     hdfs.setReplication(file12, REPLICATION);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7 + delta,
+        9 * BLOCKSIZE);
+    
     // modify file13
     hdfs.setReplication(file13, REPLICATION);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 8 + delta,
+        10 * BLOCKSIZE);
+    
     // create file14
     DFSTestUtil.createFile(hdfs, file14, BLOCKSIZE, REPLICATION, seed);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 9 + delta,
+        13 * BLOCKSIZE);
+    
     // create file15
     DFSTestUtil.createFile(hdfs, file15, BLOCKSIZE, REPLICATION, seed);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 10 + delta,
+        16 * BLOCKSIZE);
     
     // create snapshot s2 for snapshotRoot
     hdfs.createSnapshot(snapshotRoot, "s2");
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 11 + delta,
+        16 * BLOCKSIZE);
     
     // create file11 again: (0, d) + (c, 0)
     DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, REPLICATION, seed);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 12 + delta * 2,
+        19 * BLOCKSIZE);
+    
     // delete file12
     hdfs.delete(file12, true);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 13 + delta * 2,
+        19 * BLOCKSIZE);
+    
     // modify file13
     hdfs.setReplication(file13, (short) (REPLICATION - 2));
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 14 + delta * 2,
+        19 * BLOCKSIZE);
+    
     // delete file14: (c, 0) + (0, d)
     hdfs.delete(file14, true);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 15 + delta * 2,
+        19 * BLOCKSIZE);
+    
     // modify file15
-    hdfs.setReplication(file15, REP_1);
+    hdfs.setReplication(file15, REPLICATION_1);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 16 + delta * 2,
+        19 * BLOCKSIZE);
     
     // create snapshot s3 for snapshotRoot
     hdfs.createSnapshot(snapshotRoot, "s3");
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 17 + delta * 2,
+        19 * BLOCKSIZE);
+    
     // modify file10, to check if the posterior diff was set correctly
-    hdfs.setReplication(file10, REP_1);
+    hdfs.setReplication(file10, REPLICATION);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 18 + delta * 2,
+        20 * BLOCKSIZE);
     
     Path file10_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
         modDirStr + "file10");
@@ -506,6 +582,9 @@ public class TestSnapshotDeletion {
     
     // delete s2, in which process we need to combine the diff in s2 to s1
     hdfs.deleteSnapshot(snapshotRoot, "s2");
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 12 + delta,
+        14 * BLOCKSIZE);
+    
     // check the correctness of s1
     FileStatus statusAfterDeletion10 = hdfs.getFileStatus(file10_s1);
     FileStatus statusAfterDeletion11 = hdfs.getFileStatus(file11_s1);
@@ -529,10 +608,10 @@ public class TestSnapshotDeletion {
     assertFalse(hdfs.exists(file15_s1));
     
     INodeFile nodeFile13 = (INodeFile) fsdir.getINode(file13.toString());
-    assertEquals(REP_1, nodeFile13.getBlockReplication());
+    assertEquals(REPLICATION_1, nodeFile13.getBlockReplication());
 
     INodeFile nodeFile12 = (INodeFile) fsdir.getINode(file12_s1.toString());
-    assertEquals(REP_1, nodeFile12.getBlockReplication());
+    assertEquals(REPLICATION_1, nodeFile12.getBlockReplication());
   }
   
   /** Test deleting snapshots with modification on the metadata of directory */ 
@@ -544,17 +623,24 @@ public class TestSnapshotDeletion {
     
     // create snapshot s1 for sub1, and change the metadata of sub1
     SnapshotTestHelper.createSnapshot(hdfs, sub, "s1");
+    checkQuotaUsageComputation(sub, 3, BLOCKSIZE * 3);
     hdfs.setOwner(sub, "user2", "group2");
+    checkQuotaUsageComputation(sub, 3, BLOCKSIZE * 3);
     
     // create snapshot s2 for sub1, but do not modify sub1 afterwards
     hdfs.createSnapshot(sub, "s2");
+    checkQuotaUsageComputation(sub, 4, BLOCKSIZE * 3);
     
     // create snapshot s3 for sub1, and change the metadata of sub1
     hdfs.createSnapshot(sub, "s3");
+    checkQuotaUsageComputation(sub, 5, BLOCKSIZE * 3);
     hdfs.setOwner(sub, "user3", "group3");
+    checkQuotaUsageComputation(sub, 5, BLOCKSIZE * 3);
     
     // delete snapshot s3
     hdfs.deleteSnapshot(sub, "s3");
+    checkQuotaUsageComputation(sub, 4, BLOCKSIZE * 3);
+    
     // check sub1's metadata in snapshot s2
     FileStatus statusOfS2 = hdfs.getFileStatus(new Path(sub,
         HdfsConstants.DOT_SNAPSHOT_DIR + "/s2"));
@@ -563,6 +649,8 @@ public class TestSnapshotDeletion {
     
     // delete snapshot s2
     hdfs.deleteSnapshot(sub, "s2");
+    checkQuotaUsageComputation(sub, 3, BLOCKSIZE * 3);
+    
     // check sub1's metadata in snapshot s1
     FileStatus statusOfS1 = hdfs.getFileStatus(new Path(sub,
         HdfsConstants.DOT_SNAPSHOT_DIR + "/s1"));
@@ -584,22 +672,33 @@ public class TestSnapshotDeletion {
     
     // create snapshot s0 on sub
     SnapshotTestHelper.createSnapshot(hdfs, sub, "s0");
+    checkQuotaUsageComputation(sub, 5, BLOCKSIZE * 6);
     // make some changes on both sub and subsub
     final Path subFile1 = new Path(sub, "file1");
     final Path subsubFile1 = new Path(subsub, "file1");
     DFSTestUtil.createFile(hdfs, subFile1, BLOCKSIZE, REPLICATION_1, seed);
     DFSTestUtil.createFile(hdfs, subsubFile1, BLOCKSIZE, REPLICATION, seed);
+    checkQuotaUsageComputation(sub, 8, BLOCKSIZE * 11);
     
     // create snapshot s1 on sub
     SnapshotTestHelper.createSnapshot(hdfs, sub, "s1");
+    checkQuotaUsageComputation(sub, 9, BLOCKSIZE * 11);
     
     // create snapshot s2 on dir
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s2");
+    checkQuotaUsageComputation(dir, 11, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 9, BLOCKSIZE * 11);
+    
     // make changes on subsub and subsubFile1
     hdfs.setOwner(subsub, "unknown", "unknown");
     hdfs.setReplication(subsubFile1, REPLICATION_1);
+    checkQuotaUsageComputation(dir, 13, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 11, BLOCKSIZE * 11);
+    
     // make changes on sub
     hdfs.delete(subFile1, true);
+    checkQuotaUsageComputation(dir, 15, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 13, BLOCKSIZE * 11);
     
     Path subsubSnapshotCopy = SnapshotTestHelper.getSnapshotPath(dir, "s2",
         sub.getName() + Path.SEPARATOR + subsub.getName());
@@ -618,6 +717,8 @@ public class TestSnapshotDeletion {
     
     // delete snapshot s2
     hdfs.deleteSnapshot(dir, "s2");
+    checkQuotaUsageComputation(dir, 13, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 12, BLOCKSIZE * 11);
     
     // no snapshot copy for s2
     try {

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestDiff.java

@@ -86,7 +86,8 @@ public class TestDiff {
     // make modifications to current and record the diff
     final List<INode> current = new ArrayList<INode>(previous);
     
-    final List<Diff<byte[], INode>> diffs = new ArrayList<Diff<byte[], INode>>();
+    final List<Diff<byte[], INode>> diffs = 
+        new ArrayList<Diff<byte[], INode>>();
     for(int j = 0; j < 5; j++) {
       diffs.add(new Diff<byte[], INode>());
     }
@@ -246,7 +247,8 @@ public class TestDiff {
     return new INodeDirectory(n, name, PERM, 0L);
   }
 
-  static void create(INode inode, final List<INode> current, Diff<byte[], INode> diff) {
+  static void create(INode inode, final List<INode> current,
+      Diff<byte[], INode> diff) {
     final int i = Diff.search(current, inode.getKey());
     Assert.assertTrue(i < 0);
     current.add(-i - 1, inode);
@@ -272,7 +274,8 @@ public class TestDiff {
     }
   }
 
-  static void delete(INode inode, final List<INode> current, Diff<byte[], INode> diff) {
+  static void delete(INode inode, final List<INode> current,
+      Diff<byte[], INode> diff) {
     final int i = Diff.search(current, inode.getKey());
     current.remove(i);
     if (diff != null) {
@@ -297,7 +300,8 @@ public class TestDiff {
     }
   }
 
-  static void modify(INode inode, final List<INode> current, Diff<byte[], INode> diff) {
+  static void modify(INode inode, final List<INode> current,
+      Diff<byte[], INode> diff) {
     final int i = Diff.search(current, inode.getKey());
     Assert.assertTrue(i >= 0);
     final INodeDirectory oldinode = (INodeDirectory)current.get(i);