Browse Source

HDFS-6651. Deletion failure can leak inodes permanently. Contributed by Jing Zhao.

Haohui Mai 10 năm trước cách đây
mục cha
commit
7cb70d3447
21 tập tin đã thay đổi với 193 bổ sung393 xóa
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 4 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
  3. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
  4. 9 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  5. 16 32
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  6. 8 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  7. 4 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  8. 2 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
  9. 21 25
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  10. 2 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
  11. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
  12. 7 35
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  13. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
  14. 16 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
  15. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
  16. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
  17. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestAclWithSnapshot.java
  18. 1 40
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java
  19. 39 35
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  20. 52 67
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  21. 0 71
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestXAttrWithSnapshot.java

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

@@ -545,6 +545,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7696. In FsDatasetImpl, the getBlockInputStream(..) and
     getTmpInputStreams(..) methods may leak file descriptors.  (szetszwo)
 
+    HDFS-6651. Deletion failure can leak inodes permanently.
+    (Jing Zhao via wheat9)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

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

@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.util.ChunkedArrayList;
 
@@ -201,7 +200,7 @@ class FSDirDeleteOp {
    */
   private static long unprotectedDelete(
       FSDirectory fsd, INodesInPath iip, BlocksMapUpdateInfo collectedBlocks,
-      List<INode> removedINodes, long mtime) throws QuotaExceededException {
+      List<INode> removedINodes, long mtime) {
     assert fsd.hasWriteLock();
 
     // check if target node exists
@@ -234,12 +233,10 @@ class FSDirDeleteOp {
       targetNode.destroyAndCollectBlocks(collectedBlocks, removedINodes);
     } else {
       Quota.Counts counts = targetNode.cleanSubtree(CURRENT_STATE_ID,
-          latestSnapshot, collectedBlocks, removedINodes, true);
+          latestSnapshot, collectedBlocks, removedINodes);
       removed = counts.get(Quota.NAMESPACE);
-      // TODO: quota verification may fail the deletion here. We should not
-      // count the snapshot diff into quota usage in the future.
-      fsd.updateCount(iip, -counts.get(Quota.NAMESPACE),
-          -counts.get(Quota.DISKSPACE), true);
+      fsd.updateCountNoQuotaCheck(iip, iip.length() - 1,
+          -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
     }
 
     if (NameNode.stateChangeLog.isDebugEnabled()) {

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

@@ -633,7 +633,7 @@ class FSDirRenameOp {
       }
     }
 
-    boolean removeSrc4OldRename() throws IOException {
+    boolean removeSrc4OldRename() {
       final long removedSrc = fsd.removeLastINode(srcIIP);
       if (removedSrc == -1) {
         NameNode.stateChangeLog.warn("DIR* FSDirRenameOp.unprotectedRenameTo: "
@@ -648,7 +648,7 @@ class FSDirRenameOp {
       }
     }
 
-    long removeDst() throws IOException {
+    long removeDst() {
       long removedNum = fsd.removeLastINode(dstIIP);
       if (removedNum != -1) {
         oldDstChild = dstIIP.getLastINode();
@@ -735,8 +735,8 @@ class FSDirRenameOp {
         filesDeleted = true;
       } else {
         filesDeleted = oldDstChild.cleanSubtree(Snapshot.CURRENT_STATE_ID,
-            dstIIP.getLatestSnapshotId(), collectedBlocks, removedINodes,
-            true).get(Quota.NAMESPACE) >= 0;
+            dstIIP.getLatestSnapshotId(), collectedBlocks, removedINodes)
+            .get(Quota.NAMESPACE) >= 0;
       }
       fsd.getFSNamesystem().removeLeasesAndINodes(src, removedINodes, false);
       return filesDeleted;

+ 9 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -607,13 +607,12 @@ public class FSDirectory implements Closeable {
    * when image/edits have been loaded and the file/dir to be deleted is not
    * contained in snapshots.
    */
-  void updateCountForDelete(final INode inode, final INodesInPath iip)
-      throws QuotaExceededException {
+  void updateCountForDelete(final INode inode, final INodesInPath iip) {
     if (getFSNamesystem().isImageLoaded() &&
         !inode.isInLatestSnapshot(iip.getLatestSnapshotId())) {
       Quota.Counts counts = inode.computeQuotaUsage();
-      updateCount(iip, -counts.get(Quota.NAMESPACE),
-          -counts.get(Quota.DISKSPACE), false);
+      unprotectedUpdateCount(iip, iip.length() - 1,
+          -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
     }
   }
 
@@ -652,8 +651,8 @@ public class FSDirectory implements Closeable {
    * update quota of each inode and check to see if quota is exceeded. 
    * See {@link #updateCount(INodesInPath, long, long, boolean)}
    */ 
-  private void updateCountNoQuotaCheck(INodesInPath inodesInPath,
-      int numOfINodes, long nsDelta, long dsDelta) {
+  void updateCountNoQuotaCheck(INodesInPath inodesInPath, int numOfINodes,
+      long nsDelta, long dsDelta) {
     assert hasWriteLock();
     try {
       updateCount(inodesInPath, numOfINodes, nsDelta, dsDelta, false);
@@ -851,7 +850,8 @@ public class FSDirectory implements Closeable {
    * Add a child to the end of the path specified by INodesInPath.
    * @return an INodesInPath instance containing the new INode
    */
-  INodesInPath addLastINode(INodesInPath existing, INode inode,
+  @VisibleForTesting
+  public INodesInPath addLastINode(INodesInPath existing, INode inode,
       boolean checkQuota) throws QuotaExceededException {
     assert existing.getLastINode() != null &&
         existing.getLastINode().isDirectory();
@@ -926,7 +926,8 @@ public class FSDirectory implements Closeable {
    *            reference nodes;
    *          1 otherwise.
    */
-  long removeLastINode(final INodesInPath iip) throws QuotaExceededException {
+  @VisibleForTesting
+  public long removeLastINode(final INodesInPath iip) {
     final int latestSnapshot = iip.getLatestSnapshotId();
     final INode last = iip.getLastINode();
     final INodeDirectory parent = iip.getINode(-2).asDirectory();

+ 16 - 32
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.io.StringWriter;
-import java.nio.charset.Charset;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -97,8 +96,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   abstract void setUser(String user);
 
   /** Set user */
-  final INode setUser(String user, int latestSnapshotId)
-      throws QuotaExceededException {
+  final INode setUser(String user, int latestSnapshotId) {
     recordModification(latestSnapshotId);
     setUser(user);
     return this;
@@ -122,8 +120,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   abstract void setGroup(String group);
 
   /** Set group */
-  final INode setGroup(String group, int latestSnapshotId)
-      throws QuotaExceededException {
+  final INode setGroup(String group, int latestSnapshotId) {
     recordModification(latestSnapshotId);
     setGroup(group);
     return this;
@@ -148,8 +145,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   abstract void setPermission(FsPermission permission);
 
   /** Set the {@link FsPermission} of this {@link INode} */
-  INode setPermission(FsPermission permission, int latestSnapshotId) 
-      throws QuotaExceededException {
+  INode setPermission(FsPermission permission, int latestSnapshotId) {
     recordModification(latestSnapshotId);
     setPermission(permission);
     return this;
@@ -164,8 +160,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
   abstract void addAclFeature(AclFeature aclFeature);
 
-  final INode addAclFeature(AclFeature aclFeature, int latestSnapshotId)
-      throws QuotaExceededException {
+  final INode addAclFeature(AclFeature aclFeature, int latestSnapshotId) {
     recordModification(latestSnapshotId);
     addAclFeature(aclFeature);
     return this;
@@ -173,8 +168,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
   abstract void removeAclFeature();
 
-  final INode removeAclFeature(int latestSnapshotId)
-      throws QuotaExceededException {
+  final INode removeAclFeature(int latestSnapshotId) {
     recordModification(latestSnapshotId);
     removeAclFeature();
     return this;
@@ -199,8 +193,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    */
   abstract void addXAttrFeature(XAttrFeature xAttrFeature);
   
-  final INode addXAttrFeature(XAttrFeature xAttrFeature, int latestSnapshotId) 
-      throws QuotaExceededException {
+  final INode addXAttrFeature(XAttrFeature xAttrFeature, int latestSnapshotId) {
     recordModification(latestSnapshotId);
     addXAttrFeature(xAttrFeature);
     return this;
@@ -211,8 +204,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    */
   abstract void removeXAttrFeature();
   
-  final INode removeXAttrFeature(int lastestSnapshotId)
-      throws QuotaExceededException {
+  final INode removeXAttrFeature(int lastestSnapshotId) {
     recordModification(lastestSnapshotId);
     removeXAttrFeature();
     return this;
@@ -228,8 +220,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
   /** Is this inode in the latest snapshot? */
   public final boolean isInLatestSnapshot(final int latestSnapshotId) {
-    if (latestSnapshotId == Snapshot.CURRENT_STATE_ID ||
-        latestSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
+    if (latestSnapshotId == Snapshot.CURRENT_STATE_ID || latestSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
       return false;
     }
     // if parent is a reference node, parent must be a renamed node. We can 
@@ -244,15 +235,12 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     if (!parentDir.isInLatestSnapshot(latestSnapshotId)) {
       return false;
     }
-    final INode child = parentDir.getChild(getLocalNameBytes(),
-        latestSnapshotId);
+    final INode child = parentDir.getChild(getLocalNameBytes(), latestSnapshotId);
     if (this == child) {
       return true;
     }
-    if (child == null || !(child.isReference())) {
-      return false;
-    }
-    return this == child.asReference().getReferredINode();
+    return child != null && child.isReference() &&
+        this == child.asReference().getReferredINode();
   }
   
   /** @return true if the given inode is an ancestor directory of this inode. */
@@ -302,8 +290,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    *                         Note that it is {@link Snapshot#CURRENT_STATE_ID} 
    *                         if no snapshots have been taken.
    */
-  abstract void recordModification(final int latestSnapshotId)
-      throws QuotaExceededException;
+  abstract void recordModification(final int latestSnapshotId);
 
   /** Check whether it's a reference. */
   public boolean isReference() {
@@ -416,8 +403,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    */
   public abstract Quota.Counts cleanSubtree(final int snapshotId,
       int priorSnapshotId, BlocksMapUpdateInfo collectedBlocks,
-      List<INode> removedINodes, boolean countDiffChange)
-      throws QuotaExceededException;
+      List<INode> removedINodes);
   
   /**
    * Destroy self and clear everything! If the INode is a file, this method
@@ -643,15 +629,14 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   }
 
   /** Update modification time if it is larger than the current value. */
-  public abstract INode updateModificationTime(long mtime, int latestSnapshotId) 
-      throws QuotaExceededException;
+  public abstract INode updateModificationTime(long mtime, int latestSnapshotId);
 
   /** Set the last modification time of inode. */
   public abstract void setModificationTime(long modificationTime);
 
   /** Set the last modification time of inode. */
   public final INode setModificationTime(long modificationTime,
-      int latestSnapshotId) throws QuotaExceededException {
+      int latestSnapshotId) {
     recordModification(latestSnapshotId);
     setModificationTime(modificationTime);
     return this;
@@ -680,8 +665,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   /**
    * Set last access time of inode.
    */
-  public final INode setAccessTime(long accessTime, int latestSnapshotId)
-      throws QuotaExceededException {
+  public final INode setAccessTime(long accessTime, int latestSnapshotId) {
     recordModification(latestSnapshotId);
     setAccessTime(accessTime);
     return this;

+ 8 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -28,7 +28,6 @@ import java.util.Map;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
@@ -352,8 +351,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   @Override
-  public void recordModification(int latestSnapshotId)
-      throws QuotaExceededException {
+  public void recordModification(int latestSnapshotId) {
     if (isInLatestSnapshot(latestSnapshotId)
         && !shouldRecordInSrcSnapshot(latestSnapshotId)) {
       // add snapshot feature if necessary
@@ -372,7 +370,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * @return the child inode, which may be replaced.
    */
   public INode saveChild2Snapshot(final INode child, final int latestSnapshotId,
-      final INode snapshotCopy) throws QuotaExceededException {
+      final INode snapshotCopy) {
     if (latestSnapshotId == Snapshot.CURRENT_STATE_ID) {
       return child;
     }
@@ -471,8 +469,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
   /**
    * Remove the specified child from this directory.
    */
-  public boolean removeChild(INode child, int latestSnapshotId)
-      throws QuotaExceededException {
+  public boolean removeChild(INode child, int latestSnapshotId) {
     if (isInLatestSnapshot(latestSnapshotId)) {
       // create snapshot feature if necessary
       DirectoryWithSnapshotFeature sf = this.getDirectoryWithSnapshotFeature();
@@ -737,8 +734,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
   /** Call cleanSubtree(..) recursively down the subtree. */
   public Quota.Counts cleanSubtreeRecursively(final int snapshot,
       int prior, final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, final Map<INode, INode> excludedNodes, 
-      final boolean countDiffChange) throws QuotaExceededException {
+      final List<INode> removedINodes, final Map<INode, INode> excludedNodes) {
     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
@@ -753,7 +749,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
         continue;
       } else {
         Quota.Counts childCounts = child.cleanSubtree(snapshot, prior,
-            collectedBlocks, removedINodes, countDiffChange);
+            collectedBlocks, removedINodes);
         counts.add(childCounts);
       }
     }
@@ -780,13 +776,12 @@ public class INodeDirectory extends INodeWithAdditionalFields
   @Override
   public Quota.Counts cleanSubtree(final int snapshotId, int priorSnapshotId,
       final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, final boolean countDiffChange)
-      throws QuotaExceededException {
+      final List<INode> removedINodes) {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     // there is snapshot data
     if (sf != null) {
       return sf.cleanDirectory(this, snapshotId, priorSnapshotId,
-          collectedBlocks, removedINodes, countDiffChange);
+          collectedBlocks, removedINodes);
     }
     // there is no snapshot data
     if (priorSnapshotId == Snapshot.NO_SNAPSHOT_ID
@@ -799,7 +794,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
     } else {
       // process recursively down the subtree
       Quota.Counts counts = cleanSubtreeRecursively(snapshotId, priorSnapshotId,
-          collectedBlocks, removedINodes, null, countDiffChange);
+          collectedBlocks, removedINodes, null);
       if (isQuotaSet()) {
         getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
             -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));

+ 4 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -304,13 +304,11 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override
-  public void recordModification(final int latestSnapshotId)
-      throws QuotaExceededException {
+  public void recordModification(final int latestSnapshotId) {
     recordModification(latestSnapshotId, false);
   }
 
-  public void recordModification(final int latestSnapshotId, boolean withBlocks)
-      throws QuotaExceededException {
+  public void recordModification(final int latestSnapshotId, boolean withBlocks) {
     if (isInLatestSnapshot(latestSnapshotId)
         && !shouldRecordInSrcSnapshot(latestSnapshotId)) {
       // the file is in snapshot, create a snapshot feature if it does not have
@@ -489,12 +487,11 @@ public class INodeFile extends INodeWithAdditionalFields
   @Override
   public Quota.Counts cleanSubtree(final int snapshot, int priorSnapshotId,
       final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, final boolean countDiffChange)
-      throws QuotaExceededException {
+      final List<INode> removedINodes) {
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf != null) {
       return sf.cleanFile(this, snapshot, priorSnapshotId, collectedBlocks,
-          removedINodes, countDiffChange);
+          removedINodes);
     }
     Quota.Counts counts = Quota.Counts.newInstance();
     if (snapshot == CURRENT_STATE_ID) {
@@ -552,11 +549,9 @@ public class INodeFile extends INodeWithAdditionalFields
     if (sf != null) {
       FileDiffList fileDiffList = sf.getDiffs();
       int last = fileDiffList.getLastSnapshotId();
-      List<FileDiff> diffs = fileDiffList.asList();
 
       if (lastSnapshotId == Snapshot.CURRENT_STATE_ID
           || last == Snapshot.CURRENT_STATE_ID) {
-        nsDelta += diffs.size();
         dsDelta = diskspaceConsumed();
       } else if (last < lastSnapshotId) {
         dsDelta = computeFileSize(true, false) * getFileReplication();

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

@@ -22,8 +22,6 @@ import java.util.List;
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.Quota.Counts;
 import org.apache.hadoop.util.GSet;
@@ -95,8 +93,7 @@ public class INodeMap {
         "", "", new FsPermission((short) 0)), 0, 0) {
       
       @Override
-      void recordModification(int latestSnapshotId)
-          throws QuotaExceededException {
+      void recordModification(int latestSnapshotId) {
       }
       
       @Override
@@ -119,8 +116,7 @@ public class INodeMap {
       
       @Override
       public Counts cleanSubtree(int snapshotId, int priorSnapshotId,
-          BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes,
-          boolean countDiffChange) throws QuotaExceededException {
+          BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes) {
         return null;
       }
 

+ 21 - 25
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeat
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import com.google.common.base.Preconditions;
+import org.mortbay.log.Log;
 
 /**
  * An anonymous reference to an inode.
@@ -265,8 +266,7 @@ public abstract class INodeReference extends INode {
   }
   
   @Override
-  public final INode updateModificationTime(long mtime, int latestSnapshotId) 
-      throws QuotaExceededException {
+  public final INode updateModificationTime(long mtime, int latestSnapshotId) {
     return referred.updateModificationTime(mtime, latestSnapshotId);
   }
   
@@ -296,17 +296,15 @@ public abstract class INodeReference extends INode {
   }
 
   @Override
-  final void recordModification(int latestSnapshotId)
-      throws QuotaExceededException {
+  final void recordModification(int latestSnapshotId) {
     referred.recordModification(latestSnapshotId);
   }
 
   @Override // used by WithCount
   public Quota.Counts cleanSubtree(int snapshot, int prior,
-      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes,
-      final boolean countDiffChange) throws QuotaExceededException {
+      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
     return referred.cleanSubtree(snapshot, prior, collectedBlocks,
-        removedINodes, countDiffChange);
+        removedINodes);
   }
 
   @Override // used by WithCount
@@ -537,8 +535,7 @@ public abstract class INodeReference extends INode {
     @Override
     public Quota.Counts cleanSubtree(final int snapshot, int prior,
         final BlocksMapUpdateInfo collectedBlocks,
-        final List<INode> removedINodes, final boolean countDiffChange)
-        throws QuotaExceededException {
+        final List<INode> removedINodes) {
       // 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);
@@ -554,11 +551,15 @@ public abstract class INodeReference extends INode {
       }
 
       Quota.Counts counts = getReferredINode().cleanSubtree(snapshot, prior,
-          collectedBlocks, removedINodes, false);
+          collectedBlocks, removedINodes);
       INodeReference ref = getReferredINode().getParentReference();
       if (ref != null) {
-        ref.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
-            -counts.get(Quota.DISKSPACE), true);
+        try {
+          ref.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
+              -counts.get(Quota.DISKSPACE), true);
+        } catch (QuotaExceededException e) {
+          Log.warn("Should not have QuotaExceededException");
+        }
       }
       
       if (snapshot < lastSnapshotId) {
@@ -597,7 +598,7 @@ public abstract class INodeReference extends INode {
           }
           try {
             Quota.Counts counts = referred.cleanSubtree(snapshot, prior,
-                collectedBlocks, removedINodes, false);
+                collectedBlocks, removedINodes);
             INodeReference ref = getReferredINode().getParentReference();
             if (ref != null) {
               ref.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
@@ -653,8 +654,7 @@ public abstract class INodeReference extends INode {
     
     @Override
     public Quota.Counts cleanSubtree(int snapshot, int prior,
-        BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes,
-        final boolean countDiffChange) throws QuotaExceededException {
+        BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes) {
       if (snapshot == Snapshot.CURRENT_STATE_ID
           && prior == Snapshot.NO_SNAPSHOT_ID) {
         Quota.Counts counts = Quota.Counts.newInstance();
@@ -676,7 +676,7 @@ public abstract class INodeReference extends INode {
           return Quota.Counts.newInstance();
         }
         return getReferredINode().cleanSubtree(snapshot, prior,
-            collectedBlocks, removedINodes, countDiffChange);
+            collectedBlocks, removedINodes);
       }
     }
     
@@ -714,15 +714,11 @@ public abstract class INodeReference extends INode {
           Preconditions.checkState(file.isWithSnapshot());
           // make sure we mark the file as deleted
           file.getFileWithSnapshotFeature().deleteCurrentFile();
-          try {
-            // 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(snapshot, prior, collectedBlocks,
-                removedINodes, true);
-          } catch (QuotaExceededException e) {
-            LOG.error("should not exceed quota while snapshot deletion", e);
-          }
+          // 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(snapshot, prior, collectedBlocks,
+              removedINodes);
         } else if (referred.isDirectory()) {
           // similarly, if referred is a directory, it must be an
           // INodeDirectory with snapshot

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

@@ -23,10 +23,7 @@ import java.util.List;
 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.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
-import org.apache.hadoop.hdfs.server.namenode.AclFeature;
-import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
 
 /**
  * An {@link INode} representing a symbolic link.
@@ -47,7 +44,7 @@ public class INodeSymlink extends INodeWithAdditionalFields {
   }
 
   @Override
-  void recordModification(int latestSnapshotId) throws QuotaExceededException {
+  void recordModification(int latestSnapshotId) {
     if (isInLatestSnapshot(latestSnapshotId)) {
       INodeDirectory parent = getParent();
       parent.saveChild2Snapshot(this, latestSnapshotId, new INodeSymlink(this));
@@ -77,7 +74,7 @@ public class INodeSymlink extends INodeWithAdditionalFields {
   @Override
   public Quota.Counts cleanSubtree(final int snapshotId, int priorSnapshotId,
       final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, final boolean countDiffChange) {
+      final List<INode> removedINodes) {
     if (snapshotId == Snapshot.CURRENT_STATE_ID
         && priorSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
       destroyAndCollectBlocks(collectedBlocks, removedINodes);

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.LongBitFormat;
 import org.apache.hadoop.util.LightWeightGSet.LinkedElement;
@@ -239,8 +238,7 @@ public abstract class INodeWithAdditionalFields extends INode
 
   /** Update modification time if it is larger than the current value. */
   @Override
-  public final INode updateModificationTime(long mtime, int latestSnapshotId) 
-      throws QuotaExceededException {
+  public final INode updateModificationTime(long mtime, int latestSnapshotId) {
     Preconditions.checkState(isDirectory());
     if (mtime <= modificationTime) {
       return this;

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

@@ -22,8 +22,6 @@ import java.util.Collections;
 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.INodeAttributes;
@@ -71,8 +69,7 @@ abstract class AbstractINodeDiffList<N extends INode,
   public final Quota.Counts deleteSnapshotDiff(final int snapshot,
       final int prior, final N currentINode,
       final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, boolean countDiffChange) 
-      throws QuotaExceededException {
+      final List<INode> removedINodes) {
     int snapshotIndex = Collections.binarySearch(diffs, snapshot);
     
     Quota.Counts counts = Quota.Counts.newInstance();
@@ -83,14 +80,6 @@ abstract class AbstractINodeDiffList<N extends INode,
         diffs.get(snapshotIndex).setSnapshotId(prior);
       } else { // there is no snapshot before
         removed = diffs.remove(0);
-        if (countDiffChange) {
-          counts.add(Quota.NAMESPACE, 1);
-        } else {
-          // the currentINode must be a descendant of a WithName node, which set
-          // countDiffChange to false. In that case we should count in the diff
-          // change when updating the quota usage in the current tree
-          currentINode.addSpaceConsumed(-1, 0, false);
-        }
         counts.add(removed.destroyDiffAndCollectBlocks(currentINode,
             collectedBlocks, removedINodes));
       }
@@ -101,11 +90,6 @@ abstract class AbstractINodeDiffList<N extends INode,
       } else {
         // combine the to-be-removed diff with its previous diff
         removed = diffs.remove(snapshotIndex);
-        if (countDiffChange) {
-          counts.add(Quota.NAMESPACE, 1);
-        } else {
-          currentINode.addSpaceConsumed(-1, 0, false);
-        }
         if (previous.snapshotINode == null) {
           previous.snapshotINode = removed.snapshotINode;
         }
@@ -120,9 +104,7 @@ abstract class AbstractINodeDiffList<N extends INode,
   }
 
   /** Add an {@link AbstractINodeDiff} for the given snapshot. */
-  final D addDiff(int latestSnapshotId, N currentINode)
-      throws QuotaExceededException {
-    currentINode.addSpaceConsumed(1, 0, true);
+  final D addDiff(int latestSnapshotId, N currentINode) {
     return addLast(createDiff(latestSnapshotId, currentINode));
   }
 
@@ -275,26 +257,16 @@ abstract class AbstractINodeDiffList<N extends INode,
    * Check if the latest snapshot diff exists.  If not, add it.
    * @return the latest snapshot diff, which is never null.
    */
-  final D checkAndAddLatestSnapshotDiff(int latestSnapshotId, N currentINode)
-      throws QuotaExceededException {
+  final D checkAndAddLatestSnapshotDiff(int latestSnapshotId, N currentINode) {
     final D last = getLast();
-    if (last != null
-        && Snapshot.ID_INTEGER_COMPARATOR.compare(last.getSnapshotId(),
-            latestSnapshotId) >= 0) {
-      return last;
-    } else {
-      try {
-        return addDiff(latestSnapshotId, currentINode);
-      } catch(NSQuotaExceededException e) {
-        e.setMessagePrefix("Failed to record modification for snapshot");
-        throw e;
-      }
-    }
+    return (last != null && Snapshot.ID_INTEGER_COMPARATOR
+        .compare(last.getSnapshotId(), latestSnapshotId) >= 0) ?
+        last : addDiff(latestSnapshotId, currentINode);
   }
 
   /** Save the snapshot copy to the latest snapshot. */
   public D saveSelf2Snapshot(int latestSnapshotId, N currentINode,
-      A snapshotCopy) throws QuotaExceededException {
+      A snapshotCopy) {
     D diff = null;
     if (latestSnapshotId != Snapshot.CURRENT_STATE_ID) {
       diff = checkAndAddLatestSnapshotDiff(latestSnapshotId, currentINode);

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

@@ -215,7 +215,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
       int prior = Snapshot.findLatestSnapshot(snapshotRoot, snapshot.getId());
       try {
         Quota.Counts counts = snapshotRoot.cleanSubtree(snapshot.getId(),
-            prior, collectedBlocks, removedINodes, true);
+            prior, collectedBlocks, removedINodes);
         INodeDirectory parent = snapshotRoot.getParent();
         if (parent != null) {
           // there will not be any WithName node corresponding to the deleted

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

@@ -70,7 +70,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
      * Replace the given child from the created/deleted list.
      * @return true if the child is replaced; false if the child is not found.
      */
-    private final boolean replace(final ListType type,
+    private boolean replace(final ListType type,
         final INode oldChild, final INode newChild) {
       final List<INode> list = getList(type);
       final int i = search(list, oldChild.getLocalNameBytes());
@@ -83,7 +83,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
       return true;
     }
 
-    private final boolean removeChild(ListType type, final INode child) {
+    private boolean removeChild(ListType type, final INode child) {
       final List<INode> list = getList(type);
       final int i = searchIndex(type, child.getLocalNameBytes());
       if (i >= 0 && list.get(i) == child) {
@@ -410,15 +410,14 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           && snapshot != Snapshot.CURRENT_STATE_ID) {
         // this inode has been renamed before the deletion of the DstReference
         // subtree
-        inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes,
-            true);
+        inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes);
       } else { 
         // for DstReference node, continue this process to its subtree
         destroyDstSubtree(inode.asReference().getReferredINode(), snapshot,
             prior, collectedBlocks, removedINodes);
       }
     } else if (inode.isFile()) {
-      inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes, true);
+      inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes);
     } else if (inode.isDirectory()) {
       Map<INode, INode> excludedNodes = null;
       INodeDirectory dir = inode.asDirectory();
@@ -433,7 +432,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         
         if (snapshot != Snapshot.CURRENT_STATE_ID) {
           diffList.deleteSnapshotDiff(snapshot, prior, dir, collectedBlocks,
-              removedINodes, true);
+              removedINodes);
         }
         priorDiff = diffList.getDiffById(prior);
         if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
@@ -463,8 +462,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
   private static Quota.Counts cleanDeletedINode(INode inode,
       final int post, final int prior,
       final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, final boolean countDiffChange)
-      throws QuotaExceededException {
+      final List<INode> removedINodes) {
     Quota.Counts counts = Quota.Counts.newInstance();
     Deque<INode> queue = new ArrayDeque<INode>();
     queue.addLast(inode);
@@ -473,15 +471,14 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
       if (topNode instanceof INodeReference.WithName) {
         INodeReference.WithName wn = (INodeReference.WithName) topNode;
         if (wn.getLastSnapshotId() >= post) {
-          wn.cleanSubtree(post, prior, collectedBlocks, removedINodes,
-              countDiffChange);
+          wn.cleanSubtree(post, prior, collectedBlocks, removedINodes);
         }
         // 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(post, prior, file,
-            collectedBlocks, removedINodes, countDiffChange));
+            collectedBlocks, removedINodes));
       } else if (topNode.isDirectory()) {
         INodeDirectory dir = topNode.asDirectory();
         ChildrenDiff priorChildrenDiff = null;
@@ -561,7 +558,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
    * needs to make sure that parent is in the given snapshot "latest".
    */
   public boolean removeChild(INodeDirectory parent, INode child,
-      int latestSnapshotId) throws QuotaExceededException {
+      int latestSnapshotId) {
     // For a directory that is not a renamed node, if isInLatestSnapshot returns
     // false, the directory is not in the latest snapshot, thus we do not need
     // to record the removed child in any snapshot.
@@ -607,8 +604,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
   
   /** Used to record the modification of a symlink node */
   public INode saveChild2Snapshot(INodeDirectory currentINode,
-      final INode child, final int latestSnapshotId, final INode snapshotCopy)
-      throws QuotaExceededException {
+      final INode child, final int latestSnapshotId, final INode snapshotCopy) {
     Preconditions.checkArgument(!child.isDirectory(),
         "child is a directory, child=%s", child);
     Preconditions.checkArgument(latestSnapshotId != Snapshot.CURRENT_STATE_ID);
@@ -640,7 +636,6 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         deleted.computeQuotaUsage(counts, false, Snapshot.CURRENT_STATE_ID);
       }
     }
-    counts.add(Quota.NAMESPACE, diffs.asList().size());
     return counts;
   }
   
@@ -714,8 +709,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
   public Quota.Counts cleanDirectory(final INodeDirectory currentINode,
       final int snapshot, int prior,
       final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, final boolean countDiffChange)
-      throws QuotaExceededException {
+      final List<INode> removedINodes) {
     Quota.Counts counts = Quota.Counts.newInstance();
     Map<INode, INode> priorCreated = null;
     Map<INode, INode> priorDeleted = null;
@@ -728,7 +722,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
             collectedBlocks, removedINodes));
       }
       counts.add(currentINode.cleanSubtreeRecursively(snapshot, prior,
-          collectedBlocks, removedINodes, priorDeleted, countDiffChange));
+          collectedBlocks, removedINodes, priorDeleted));
     } else {
       // update prior
       prior = getDiffs().updatePrior(snapshot, prior);
@@ -745,9 +739,9 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
       }
       
       counts.add(getDiffs().deleteSnapshotDiff(snapshot, prior,
-          currentINode, collectedBlocks, removedINodes, countDiffChange));
+          currentINode, collectedBlocks, removedINodes));
       counts.add(currentINode.cleanSubtreeRecursively(snapshot, prior,
-          collectedBlocks, removedINodes, priorDeleted, countDiffChange));
+          collectedBlocks, removedINodes, priorDeleted));
 
       // check priorDiff again since it may be created during the diff deletion
       if (prior != Snapshot.NO_SNAPSHOT_ID) {
@@ -764,7 +758,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
                 ListType.CREATED)) {
               if (priorCreated.containsKey(cNode)) {
                 counts.add(cNode.cleanSubtree(snapshot, Snapshot.NO_SNAPSHOT_ID,
-                    collectedBlocks, removedINodes, countDiffChange));
+                    collectedBlocks, removedINodes));
               }
             }
           }
@@ -781,7 +775,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
               ListType.DELETED)) {
             if (priorDeleted == null || !priorDeleted.containsKey(dNode)) {
               counts.add(cleanDeletedINode(dNode, snapshot, prior,
-                  collectedBlocks, removedINodes, countDiffChange));
+                  collectedBlocks, removedINodes));
             }
           }
         }

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

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -48,8 +47,7 @@ public class FileDiffList extends
   }
 
   public void saveSelf2Snapshot(int latestSnapshotId, INodeFile iNodeFile,
-      INodeFileAttributes snapshotCopy, boolean withBlocks)
-          throws QuotaExceededException {
+      INodeFileAttributes snapshotCopy, boolean withBlocks) {
     final FileDiff diff =
         super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy);
     if(withBlocks)  // Store blocks if this is the first update

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

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -117,8 +116,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
   
   public Quota.Counts cleanFile(final INodeFile file, final int snapshotId,
       int priorSnapshotId, final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, final boolean countDiffChange)
-      throws QuotaExceededException {
+      final List<INode> removedINodes) {
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       // delete the current file while the file has snapshot feature
       if (!isCurrentFileDeleted()) {
@@ -130,7 +128,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
     } else { // delete the snapshot
       priorSnapshotId = getDiffs().updatePrior(snapshotId, priorSnapshotId);
       return diffs.deleteSnapshotDiff(snapshotId, priorSnapshotId, file,
-          collectedBlocks, removedINodes, countDiffChange);
+          collectedBlocks, removedINodes);
     }
   }
   

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

@@ -38,7 +38,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.AclStorage;
@@ -611,7 +610,6 @@ public class TestAclWithSnapshot {
 
     aclSpec = Lists.newArrayList(
       aclEntry(ACCESS, USER, "bruce", READ));
-    exception.expect(NSQuotaExceededException.class);
     hdfs.modifyAclEntries(filePath, aclSpec);
   }
 
@@ -647,7 +645,6 @@ public class TestAclWithSnapshot {
 
     aclSpec = Lists.newArrayList(
       aclEntry(ACCESS, USER, "bruce", READ));
-    exception.expect(NSQuotaExceededException.class);
     hdfs.removeAcl(filePath);
   }
 

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

@@ -34,13 +34,11 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
-import org.apache.hadoop.ipc.RemoteException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -239,7 +237,7 @@ public class TestNestedSnapshots {
   }
 
   @Test (timeout=300000)
-  public void testSnapshotWithQuota() throws Exception {
+  public void testSnapshotName() throws Exception {
     final String dirStr = "/testSnapshotWithQuota/dir";
     final Path dir = new Path(dirStr);
     hdfs.mkdirs(dir, new FsPermission((short)0777));
@@ -266,43 +264,6 @@ public class TestNestedSnapshots {
       Assert.assertEquals(HdfsConstants.DOT_SNAPSHOT_DIR, parent.getName());
       Assert.assertEquals(dir, parent.getParent());
     }
-    final Path f2 = new Path(foo, "f2");
-    DFSTestUtil.createFile(hdfs, f2, BLOCKSIZE, REPLICATION, SEED);
-    
-    try {
-      // normal create file should fail with quota
-      final Path f3 = new Path(foo, "f3");
-      DFSTestUtil.createFile(hdfs, f3, BLOCKSIZE, REPLICATION, SEED);
-      Assert.fail();
-    } catch(NSQuotaExceededException e) {
-      SnapshotTestHelper.LOG.info("The exception is expected.", e);
-    }
-
-    try {
-      // createSnapshot should fail with quota
-      hdfs.createSnapshot(dir);
-      Assert.fail();
-    } catch(NSQuotaExceededException e) {
-      SnapshotTestHelper.LOG.info("The exception is expected.", e);
-    }
-
-    try {
-      // setPermission f1 should fail with quote since it cannot add diff.
-      hdfs.setPermission(f1, new FsPermission((short)0));
-      Assert.fail();
-    } catch(RemoteException e) {
-      Assert.assertSame(NSQuotaExceededException.class,
-          e.unwrapRemoteException().getClass());
-      SnapshotTestHelper.LOG.info("The exception is expected.", e);
-    }
-
-    // setPermission f2 since it was created after the snapshot
-    hdfs.setPermission(f2, new FsPermission((short)0));
-
-    // increase quota and retry the commands.
-    hdfs.setQuota(dir, NS_QUOTA + 2, HdfsConstants.QUOTA_DONT_SET);
-    hdfs.createSnapshot(dir, "s1");
-    hdfs.setPermission(foo, new FsPermission((short)0444));
   }
 
   /**

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

@@ -50,7 +50,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
@@ -62,6 +62,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
+import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
@@ -73,6 +74,7 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
+import org.mockito.internal.util.reflection.Whitebox;
 
 /** Testing rename with snapshots. */
 public class TestRenameWithSnapshots {
@@ -1197,7 +1199,7 @@ public class TestRenameWithSnapshots {
     restartClusterAndCheckImage(true);
     // make sure the whole referred subtree has been destroyed
     Quota.Counts q = fsdir.getRoot().getDirectoryWithQuotaFeature().getSpaceConsumed();  
-    assertEquals(4, q.get(Quota.NAMESPACE));
+    assertEquals(3, q.get(Quota.NAMESPACE));
     assertEquals(0, q.get(Quota.DISKSPACE));
     
     hdfs.deleteSnapshot(sdir1, "s1");
@@ -1558,29 +1560,34 @@ public class TestRenameWithSnapshots {
     SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
     SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
     
-    // set ns quota of dir2 to 5, so the current remaining is 2 (already has
-    // dir2, subdir2, and s2)
-    hdfs.setQuota(dir2, 5, Long.MAX_VALUE - 1);
+    // set ns quota of dir2 to 4, so the current remaining is 2 (already has
+    // dir2, and subdir2)
+    hdfs.setQuota(dir2, 4, Long.MAX_VALUE - 1);
     
     final Path foo2 = new Path(subdir2, foo.getName());
+    FSDirectory fsdir2 = Mockito.spy(fsdir);
+    Mockito.doThrow(new NSQuotaExceededException("fake exception")).when(fsdir2)
+        .addLastINode((INodesInPath) Mockito.anyObject(),
+            (INode) Mockito.anyObject(), Mockito.anyBoolean());
+    Whitebox.setInternalState(fsn, "dir", fsdir2);
     // rename /test/dir1/foo to /test/dir2/subdir2/foo. 
-    // FSDirectory#verifyQuota4Rename will pass since foo/bar only be counted 
-    // as 2 in NS quota. However, the rename operation will fail when adding
-    // foo to subdir2, since we will create a snapshot diff for subdir2. 
+    // FSDirectory#verifyQuota4Rename will pass since the remaining quota is 2.
+    // However, the rename operation will fail since we let addLastINode throw
+    // NSQuotaExceededException
     boolean rename = hdfs.rename(foo, foo2);
     assertFalse(rename);
     
     // check the undo
     assertTrue(hdfs.exists(foo));
     assertTrue(hdfs.exists(bar));
-    INodeDirectory dir1Node = fsdir.getINode4Write(dir1.toString())
+    INodeDirectory dir1Node = fsdir2.getINode4Write(dir1.toString())
         .asDirectory();
     List<INode> childrenList = ReadOnlyList.Util.asList(dir1Node
         .getChildrenList(Snapshot.CURRENT_STATE_ID));
     assertEquals(1, childrenList.size());
     INode fooNode = childrenList.get(0);
     assertTrue(fooNode.asDirectory().isWithSnapshot());
-    INode barNode = fsdir.getINode4Write(bar.toString());
+    INode barNode = fsdir2.getINode4Write(bar.toString());
     assertTrue(barNode.getClass() == INodeFile.class);
     assertSame(fooNode, barNode.getParent());
     List<DirectoryDiff> diffList = dir1Node
@@ -1591,17 +1598,17 @@ public class TestRenameWithSnapshots {
     assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
     
     // check dir2
-    INodeDirectory dir2Node = fsdir.getINode4Write(dir2.toString()).asDirectory();
+    INodeDirectory dir2Node = fsdir2.getINode4Write(dir2.toString()).asDirectory();
     assertTrue(dir2Node.isSnapshottable());
     Quota.Counts counts = dir2Node.computeQuotaUsage();
-    assertEquals(3, counts.get(Quota.NAMESPACE));
+    assertEquals(2, counts.get(Quota.NAMESPACE));
     assertEquals(0, counts.get(Quota.DISKSPACE));
     childrenList = ReadOnlyList.Util.asList(dir2Node.asDirectory()
         .getChildrenList(Snapshot.CURRENT_STATE_ID));
     assertEquals(1, childrenList.size());
     INode subdir2Node = childrenList.get(0);
     assertSame(dir2Node, subdir2Node.getParent());
-    assertSame(subdir2Node, fsdir.getINode4Write(subdir2.toString()));
+    assertSame(subdir2Node, fsdir2.getINode4Write(subdir2.toString()));
     diffList = dir2Node.getDiffs().asList();
     assertEquals(1, diffList.size());
     diff = diffList.get(0);
@@ -1628,27 +1635,28 @@ public class TestRenameWithSnapshots {
     SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
     SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
     
-    // set ns quota of dir2 to 4, so the current remaining is 0 (already has
-    // dir2, sub_dir2, subsub_dir2, and s2)
+    // set ns quota of dir2 to 4, so the current remaining is 1 (already has
+    // dir2, sub_dir2, and subsub_dir2)
     hdfs.setQuota(dir2, 4, Long.MAX_VALUE - 1);
-    
+    FSDirectory fsdir2 = Mockito.spy(fsdir);
+    Mockito.doThrow(new RuntimeException("fake exception")).when(fsdir2)
+        .removeLastINode((INodesInPath) Mockito.anyObject());
+    Whitebox.setInternalState(fsn, "dir", fsdir2);
     // rename /test/dir1/foo to /test/dir2/sub_dir2/subsub_dir2. 
     // FSDirectory#verifyQuota4Rename will pass since foo only be counted 
     // as 1 in NS quota. However, the rename operation will fail when removing
-    // subsub_dir2 since this step tries to add a snapshot diff in sub_dir2.
+    // subsub_dir2.
     try {
       hdfs.rename(foo, subsub_dir2, Rename.OVERWRITE);
       fail("Expect QuotaExceedException");
-    } catch (QuotaExceededException e) {
-      String msg = "Failed to record modification for snapshot: "
-          + "The NameSpace quota (directories and files)"
-          + " is exceeded: quota=4 file count=5"; 
+    } catch (Exception e) {
+      String msg = "fake exception";
       GenericTestUtils.assertExceptionContains(msg, e);
     }
     
     // check the undo
     assertTrue(hdfs.exists(foo));
-    INodeDirectory dir1Node = fsdir.getINode4Write(dir1.toString())
+    INodeDirectory dir1Node = fsdir2.getINode4Write(dir1.toString())
         .asDirectory();
     List<INode> childrenList = ReadOnlyList.Util.asList(dir1Node
         .getChildrenList(Snapshot.CURRENT_STATE_ID));
@@ -1664,19 +1672,18 @@ public class TestRenameWithSnapshots {
     assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
     
     // check dir2
-    INodeDirectory dir2Node = fsdir.getINode4Write(dir2.toString()).asDirectory();
+    INodeDirectory dir2Node = fsdir2.getINode4Write(dir2.toString()).asDirectory();
     assertTrue(dir2Node.isSnapshottable());
     Quota.Counts counts = dir2Node.computeQuotaUsage();
-    assertEquals(4, counts.get(Quota.NAMESPACE));
+    assertEquals(3, counts.get(Quota.NAMESPACE));
     assertEquals(0, counts.get(Quota.DISKSPACE));
     childrenList = ReadOnlyList.Util.asList(dir2Node.asDirectory()
         .getChildrenList(Snapshot.CURRENT_STATE_ID));
     assertEquals(1, childrenList.size());
     INode subdir2Node = childrenList.get(0);
-    assertTrue(subdir2Node.asDirectory().isWithSnapshot());
     assertSame(dir2Node, subdir2Node.getParent());
-    assertSame(subdir2Node, fsdir.getINode4Write(sub_dir2.toString()));
-    INode subsubdir2Node = fsdir.getINode4Write(subsub_dir2.toString());
+    assertSame(subdir2Node, fsdir2.getINode4Write(sub_dir2.toString()));
+    INode subsubdir2Node = fsdir2.getINode4Write(subsub_dir2.toString());
     assertTrue(subsubdir2Node.getClass() == INodeDirectory.class);
     assertSame(subdir2Node, subsubdir2Node.getParent());
     
@@ -1685,9 +1692,6 @@ public class TestRenameWithSnapshots {
     diff = diffList.get(0);
     assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
     assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
-    
-    diffList = subdir2Node.asDirectory().getDiffs().asList();
-    assertEquals(0, diffList.size());
   }
   
   /**
@@ -1787,7 +1791,7 @@ public class TestRenameWithSnapshots {
     INode dir2Node = fsdir.getINode4Write(dir2.toString());
     assertTrue(dir2Node.asDirectory().isSnapshottable());
     Quota.Counts counts = dir2Node.computeQuotaUsage();
-    assertEquals(7, counts.get(Quota.NAMESPACE));
+    assertEquals(4, counts.get(Quota.NAMESPACE));
     assertEquals(BLOCKSIZE * REPL * 2, counts.get(Quota.DISKSPACE));
   }
   
@@ -1955,11 +1959,11 @@ public class TestRenameWithSnapshots {
     final INodeDirectory dir1Node = fsdir.getINode4Write(sdir1.toString())
         .asDirectory();
     Quota.Counts q1 = dir1Node.getDirectoryWithQuotaFeature().getSpaceConsumed();  
-    assertEquals(4, q1.get(Quota.NAMESPACE));
+    assertEquals(3, q1.get(Quota.NAMESPACE));
     final INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
         .asDirectory();
     Quota.Counts q2 = dir2Node.getDirectoryWithQuotaFeature().getSpaceConsumed();  
-    assertEquals(2, q2.get(Quota.NAMESPACE));
+    assertEquals(1, q2.get(Quota.NAMESPACE));
     
     final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
         foo.getName());
@@ -2025,11 +2029,11 @@ public class TestRenameWithSnapshots {
         .asDirectory();
     // sdir1 + s1 + foo_s1 (foo) + foo (foo + s1 + bar~bar3)
     Quota.Counts q1 = dir1Node.getDirectoryWithQuotaFeature().getSpaceConsumed();  
-    assertEquals(9, q1.get(Quota.NAMESPACE));
+    assertEquals(7, q1.get(Quota.NAMESPACE));
     final INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
         .asDirectory();
     Quota.Counts q2 = dir2Node.getDirectoryWithQuotaFeature().getSpaceConsumed();  
-    assertEquals(2, q2.get(Quota.NAMESPACE));
+    assertEquals(1, q2.get(Quota.NAMESPACE));
     
     final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
         foo.getName());

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

@@ -263,7 +263,7 @@ public class TestSnapshotDeletion {
     BlockInfo[] blocks = temp.getBlocks();
     hdfs.delete(tempDir, true);
     // check dir's quota usage
-    checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 3);
+    checkQuotaUsageComputation(dir, 8, BLOCKSIZE * REPLICATION * 3);
     // check blocks of tempFile
     for (BlockInfo b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
@@ -279,7 +279,7 @@ public class TestSnapshotDeletion {
     // create snapshot s1
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
     // check dir's quota usage
-    checkQuotaUsageComputation(dir, 14L, BLOCKSIZE * REPLICATION * 4);
+    checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4);
     
     // get two snapshots for later use
     Snapshot snapshot0 = fsdir.getINode(dir.toString()).asDirectory()
@@ -295,7 +295,7 @@ public class TestSnapshotDeletion {
     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);
+    checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4);
     
     // check the snapshot copy of noChangeDir 
     Path snapshotNoChangeDir = SnapshotTestHelper.getSnapshotPath(dir, "s1",
@@ -337,11 +337,11 @@ public class TestSnapshotDeletion {
     final INodeFile newFileNode = TestSnapshotBlocksMap.assertBlockCollection(
         newFile.toString(), 1, fsdir, blockmanager);
     blocks = newFileNode.getBlocks();
-    checkQuotaUsageComputation(dir, 18L, BLOCKSIZE * REPLICATION * 5);
+    checkQuotaUsageComputation(dir, 10L, BLOCKSIZE * REPLICATION * 5);
     hdfs.delete(sub, true);
     // while deletion, we add diff for subsub and metaChangeFile1, and remove
     // newFile
-    checkQuotaUsageComputation(dir, 19L, BLOCKSIZE * REPLICATION * 4);
+    checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4);
     for (BlockInfo b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
@@ -426,13 +426,13 @@ public class TestSnapshotDeletion {
     // create snapshot s1 for sub
     SnapshotTestHelper.createSnapshot(hdfs, sub, snapshotName);
     // check quota usage computation
-    checkQuotaUsageComputation(sub, 4, BLOCKSIZE * REPLICATION * 2);
+    checkQuotaUsageComputation(sub, 3, 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);
+    checkQuotaUsageComputation(sub, 3, BLOCKSIZE * REPLICATION * 2);
     
     // create a new file under sub
     Path newFile = new Path(sub, "newFile");
@@ -440,14 +440,14 @@ public class TestSnapshotDeletion {
     // create another snapshot s2
     String snapshotName2 = "s2";
     hdfs.createSnapshot(sub, snapshotName2);
-    checkQuotaUsageComputation(sub, 6, BLOCKSIZE * REPLICATION * 3);
+    checkQuotaUsageComputation(sub, 4, 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);
+    checkQuotaUsageComputation(sub, 4, BLOCKSIZE * REPLICATION * 3);
     FileStatus statusAfterDeletion = hdfs.getFileStatus(ss);
     System.out.println("Before deletion: " + statusBeforeDeletion.toString()
         + "\n" + "After deletion: " + statusAfterDeletion.toString());
@@ -483,28 +483,28 @@ public class TestSnapshotDeletion {
     
     // create snapshot s0 on dir
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
-    checkQuotaUsageComputation(dir, 8, 3 * BLOCKSIZE * REPLICATION);
+    checkQuotaUsageComputation(dir, 7, 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);
+    checkQuotaUsageComputation(dir, 7, 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);
+    checkQuotaUsageComputation(dir, 7, 3 * BLOCKSIZE * REPLICATION);
     
     // create snapshot s1 on dir
     hdfs.createSnapshot(dir, "s1");
-    checkQuotaUsageComputation(dir, 12, 3 * BLOCKSIZE * REPLICATION);
+    checkQuotaUsageComputation(dir, 7, 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);
+    checkQuotaUsageComputation(dir, 6, 2 * BLOCKSIZE * REPLICATION - BLOCKSIZE);
     for (BlockInfo b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
@@ -679,21 +679,21 @@ public class TestSnapshotDeletion {
     // create another snapshot
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s2");
     
-    checkQuotaUsageComputation(dir, 11, BLOCKSIZE * 2 * REPLICATION);
+    checkQuotaUsageComputation(dir, 7, BLOCKSIZE * 2 * REPLICATION);
     
     // delete subsubdir and subDir2
     hdfs.delete(subsubDir, true);
     hdfs.delete(subDir2, true);
     
     // add diff of s2 to subDir1, subsubDir, and subDir2
-    checkQuotaUsageComputation(dir, 14, BLOCKSIZE * 2 * REPLICATION);
+    checkQuotaUsageComputation(dir, 7, BLOCKSIZE * 2 * REPLICATION);
     
     // delete snapshot s2
     hdfs.deleteSnapshot(dir, "s2");
     
     // delete s2 diff in dir, subDir2, and subsubDir. Delete newFile, newDir,
     // and newFile2. Rename s2 diff to s1 for subDir1 
-    checkQuotaUsageComputation(dir, 8, 0);
+    checkQuotaUsageComputation(dir, 4, 0);
     // Check rename of snapshot diff in subDir1
     Path subdir1_s1 = SnapshotTestHelper.getSnapshotPath(dir, "s1",
         subDir1.getName());
@@ -714,7 +714,6 @@ public class TestSnapshotDeletion {
       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");
@@ -728,72 +727,59 @@ public class TestSnapshotDeletion {
 
     // create snapshot s1 for snapshotRoot
     SnapshotTestHelper.createSnapshot(hdfs, snapshotRoot, "s1");
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 5, 8 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 4, 8 * BLOCKSIZE);
     
     // delete file11
     hdfs.delete(file11, true);
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 6 + delta,
-        8 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 4, 8 * BLOCKSIZE);
     
     // modify file12
     hdfs.setReplication(file12, REPLICATION);
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7 + delta,
-        9 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 4, 9 * BLOCKSIZE);
     
     // modify file13
     hdfs.setReplication(file13, REPLICATION);
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 8 + delta,
-        10 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 4, 10 * BLOCKSIZE);
     
     // create file14
     DFSTestUtil.createFile(hdfs, file14, BLOCKSIZE, REPLICATION, seed);
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 9 + delta,
-        13 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 5, 13 * BLOCKSIZE);
     
     // create file15
     DFSTestUtil.createFile(hdfs, file15, BLOCKSIZE, REPLICATION, seed);
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 10 + delta,
-        16 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 6, 16 * BLOCKSIZE);
     
     // create snapshot s2 for snapshotRoot
     hdfs.createSnapshot(snapshotRoot, "s2");
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 11 + delta,
-        16 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 6, 16 * BLOCKSIZE);
     
     // create file11 again: (0, d) + (c, 0)
     DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, REPLICATION, seed);
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 12 + delta * 2,
-        19 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 19 * BLOCKSIZE);
     
     // delete file12
     hdfs.delete(file12, true);
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 13 + delta * 2,
-        19 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 19 * BLOCKSIZE);
     
     // modify file13
     hdfs.setReplication(file13, (short) (REPLICATION - 2));
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 14 + delta * 2,
-        19 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 19 * BLOCKSIZE);
     
     // delete file14: (c, 0) + (0, d)
     hdfs.delete(file14, true);
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 15 + delta * 2,
-        19 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 19 * BLOCKSIZE);
     
     // modify file15
     hdfs.setReplication(file15, REPLICATION_1);
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 16 + delta * 2,
-        19 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 19 * BLOCKSIZE);
     
     // create snapshot s3 for snapshotRoot
     hdfs.createSnapshot(snapshotRoot, "s3");
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 17 + delta * 2,
-        19 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 19 * BLOCKSIZE);
     
     // modify file10, to check if the posterior diff was set correctly
     hdfs.setReplication(file10, REPLICATION);
-    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 18 + delta * 2,
-        20 * BLOCKSIZE);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 20 * BLOCKSIZE);
     
     Path file10_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
         modDirStr + "file10");
@@ -819,8 +805,7 @@ 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);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 6, 14 * BLOCKSIZE);
     
     // check the correctness of s1
     FileStatus statusAfterDeletion10 = hdfs.getFileStatus(file10_s1);
@@ -873,23 +858,23 @@ public class TestSnapshotDeletion {
     
     // create snapshot s1 for sub1, and change the metadata of sub1
     SnapshotTestHelper.createSnapshot(hdfs, sub, "s1");
-    checkQuotaUsageComputation(sub, 3, BLOCKSIZE * 3);
+    checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
     hdfs.setOwner(sub, "user2", "group2");
-    checkQuotaUsageComputation(sub, 3, BLOCKSIZE * 3);
+    checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
     
     // create snapshot s2 for sub1, but do not modify sub1 afterwards
     hdfs.createSnapshot(sub, "s2");
-    checkQuotaUsageComputation(sub, 4, BLOCKSIZE * 3);
+    checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
     
     // create snapshot s3 for sub1, and change the metadata of sub1
     hdfs.createSnapshot(sub, "s3");
-    checkQuotaUsageComputation(sub, 5, BLOCKSIZE * 3);
+    checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
     hdfs.setOwner(sub, "user3", "group3");
-    checkQuotaUsageComputation(sub, 5, BLOCKSIZE * 3);
+    checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
     
     // delete snapshot s3
     hdfs.deleteSnapshot(sub, "s3");
-    checkQuotaUsageComputation(sub, 4, BLOCKSIZE * 3);
+    checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
     
     // check sub1's metadata in snapshot s2
     FileStatus statusOfS2 = hdfs.getFileStatus(new Path(sub,
@@ -899,7 +884,7 @@ public class TestSnapshotDeletion {
     
     // delete snapshot s2
     hdfs.deleteSnapshot(sub, "s2");
-    checkQuotaUsageComputation(sub, 3, BLOCKSIZE * 3);
+    checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
     
     // check sub1's metadata in snapshot s1
     FileStatus statusOfS1 = hdfs.getFileStatus(new Path(sub,
@@ -957,34 +942,34 @@ public class TestSnapshotDeletion {
     
     // create snapshot s0 on sub
     SnapshotTestHelper.createSnapshot(hdfs, sub, "s0");
-    checkQuotaUsageComputation(sub, 5, BLOCKSIZE * 6);
+    checkQuotaUsageComputation(sub, 4, 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);
+    checkQuotaUsageComputation(sub, 6, BLOCKSIZE * 11);
     
     // create snapshot s1 on sub
     SnapshotTestHelper.createSnapshot(hdfs, sub, "s1");
-    checkQuotaUsageComputation(sub, 9, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 6, BLOCKSIZE * 11);
     
     // create snapshot s2 on dir
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s2");
-    checkQuotaUsageComputation(dir, 11, BLOCKSIZE * 11);
-    checkQuotaUsageComputation(sub, 9, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(dir, 7, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 6, 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);
+    checkQuotaUsageComputation(dir, 7, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 6, BLOCKSIZE * 11);
     
     // make changes on sub
     hdfs.delete(subFile1, true);
-    checkQuotaUsageComputation(new Path("/"), 16, BLOCKSIZE * 11);
-    checkQuotaUsageComputation(dir, 15, BLOCKSIZE * 11);
-    checkQuotaUsageComputation(sub, 13, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(new Path("/"), 8, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(dir, 7, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 6, BLOCKSIZE * 11);
     
     Path subsubSnapshotCopy = SnapshotTestHelper.getSnapshotPath(dir, "s2",
         sub.getName() + Path.SEPARATOR + subsub.getName());
@@ -1003,9 +988,9 @@ public class TestSnapshotDeletion {
     
     // delete snapshot s2
     hdfs.deleteSnapshot(dir, "s2");
-    checkQuotaUsageComputation(new Path("/"), 14, BLOCKSIZE * 11);
-    checkQuotaUsageComputation(dir, 13, BLOCKSIZE * 11);
-    checkQuotaUsageComputation(sub, 12, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(new Path("/"), 8, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(dir, 7, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 6, BLOCKSIZE * 11);
     
     // no snapshot copy for s2
     try {

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

@@ -343,77 +343,6 @@ public class TestXAttrWithSnapshot {
     hdfs.removeXAttr(snapshotPath, name1);
   }
 
-  /**
-   * Assert exception of setting xattr when exceeding quota.
-   */
-  @Test
-  public void testSetXAttrExceedsQuota() throws Exception {
-    Path filePath = new Path(path, "file1");
-    Path fileSnapshotPath = new Path(snapshotPath, "file1");
-    FileSystem.mkdirs(hdfs, path, FsPermission.createImmutable((short) 0755));
-    hdfs.allowSnapshot(path);
-    hdfs.setQuota(path, 3, HdfsConstants.QUOTA_DONT_SET);
-    FileSystem.create(hdfs, filePath,
-        FsPermission.createImmutable((short) 0600)).close();
-    hdfs.setXAttr(filePath, name1, value1);
-
-    hdfs.createSnapshot(path, snapshotName);
-
-    byte[] value = hdfs.getXAttr(filePath, name1);
-    Assert.assertArrayEquals(value, value1);
-
-    value = hdfs.getXAttr(fileSnapshotPath, name1);
-    Assert.assertArrayEquals(value, value1);
-
-    exception.expect(NSQuotaExceededException.class);
-    hdfs.setXAttr(filePath, name2, value2);
-  }
-
-
-  /**
-   * Test that an exception is thrown when adding an XAttr Feature to
-   * a snapshotted path
-   */
-  @Test
-  public void testSetXAttrAfterSnapshotExceedsQuota() throws Exception {
-    Path filePath = new Path(path, "file1");
-    FileSystem.mkdirs(hdfs, path, FsPermission.createImmutable((short) 0755));
-    hdfs.allowSnapshot(path);
-    hdfs.setQuota(path, 3, HdfsConstants.QUOTA_DONT_SET);
-    FileSystem.create(hdfs, filePath,
-        FsPermission.createImmutable((short) 0600)).close();
-    hdfs.createSnapshot(path, snapshotName);
-    // This adds an XAttr feature, which can throw an exception
-    exception.expect(NSQuotaExceededException.class);
-    hdfs.setXAttr(filePath, name1, value1);
-  }
-
-  /**
-   * Assert exception of removing xattr when exceeding quota.
-   */
-  @Test
-  public void testRemoveXAttrExceedsQuota() throws Exception {
-    Path filePath = new Path(path, "file1");
-    Path fileSnapshotPath = new Path(snapshotPath, "file1");
-    FileSystem.mkdirs(hdfs, path, FsPermission.createImmutable((short) 0755));
-    hdfs.allowSnapshot(path);
-    hdfs.setQuota(path, 3, HdfsConstants.QUOTA_DONT_SET);
-    FileSystem.create(hdfs, filePath,
-        FsPermission.createImmutable((short) 0600)).close();
-    hdfs.setXAttr(filePath, name1, value1);
-
-    hdfs.createSnapshot(path, snapshotName);
-
-    byte[] value = hdfs.getXAttr(filePath, name1);
-    Assert.assertArrayEquals(value, value1);
-
-    value = hdfs.getXAttr(fileSnapshotPath, name1);
-    Assert.assertArrayEquals(value, value1);
-
-    exception.expect(NSQuotaExceededException.class);
-    hdfs.removeXAttr(filePath, name1);
-  }
-
   /**
    * Test that users can copy a snapshot while preserving its xattrs.
    */