Browse Source

HDFS-7723. Quota By Storage Type namenode implemenation. (Contributed by Xiaoyu Yao)

Arpit Agarwal 10 years ago
parent
commit
676fc2d1dc
52 changed files with 2160 additions and 719 deletions
  1. 3 3
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  3. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
  4. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryComputationContext.java
  5. 163 65
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DirectoryWithQuotaFeature.java
  6. 46 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
  7. 32 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
  8. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
  9. 24 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
  10. 125 46
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  11. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  12. 12 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  13. 85 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  14. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
  15. 40 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  16. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  17. 72 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
  18. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  19. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  20. 46 30
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  21. 61 47
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  22. 15 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryAttributes.java
  23. 63 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  24. 10 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
  25. 41 38
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  26. 11 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
  27. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
  28. 2 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  29. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Quota.java
  30. 85 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/QuotaByStorageTypeEntry.java
  31. 179 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/QuotaCounts.java
  32. 8 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
  33. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  34. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
  35. 57 46
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
  36. 30 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
  37. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
  38. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
  39. 56 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
  40. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
  41. 33 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/EnumCounters.java
  42. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
  43. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  44. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
  45. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
  46. 17 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java
  47. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
  48. 524 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaByStorageType.java
  49. 25 24
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  50. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  51. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
  52. 191 182
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

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

@@ -27,14 +27,14 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7655. Expose truncate API for Web HDFS. (yliu)
 
-    HDFS-7720. Quota by Storage Type API, tools and ClientNameNode Protocol
-    changes. (Xiaoyu Yao via Arpit Agarwal)
-
     HDFS-6133. Add a feature for replica pinning so that a pinned replica
     will not be moved by Balancer/Mover.  (zhaoyunjiong via szetszwo)
 
     HDFS-3689. Add support for variable length block. Contributed by Jing Zhao.
 
+    HDFS-7584. Enable Quota Support for Storage Types. (Xiaoyu Yao via
+    Arpit Agarwal)
+
   IMPROVEMENTS
 
     HDFS-7055. Add tracing to DFSInputStream (cmccabe)

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -421,6 +421,10 @@ public class BlockManager {
     }
   }
 
+  public BlockStoragePolicySuite getStoragePolicySuite() {
+    return storagePolicySuite;
+  }
+
   /** get the BlockTokenSecretManager */
   @VisibleForTesting
   public BlockTokenSecretManager getBlockTokenSecretManager() {

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

@@ -229,7 +229,9 @@ public class BackupImage extends FSImage {
       }
       lastAppliedTxId = logLoader.getLastAppliedTxId();
 
-      FSImage.updateCountForQuota(getNamesystem().dir.getRoot()); // inefficient!
+      FSImage.updateCountForQuota(
+          getNamesystem().dir.getBlockStoragePolicySuite(),
+          getNamesystem().dir.rootDir); // inefficient!
     } finally {
       backupInputStream.clear();
     }

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryComputationContext.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
@@ -116,4 +117,8 @@ public class ContentSummaryComputationContext {
   public Content.Counts getCounts() {
     return counts;
   }
+
+  public BlockStoragePolicySuite getBlockStoragePolicySuite() {
+      return fsn.getBlockManager().getStoragePolicySuite();
+  }
 }

+ 163 - 65
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DirectoryWithQuotaFeature.java

@@ -21,47 +21,97 @@ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 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.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.util.EnumCounters;
 
 /**
  * Quota feature for {@link INodeDirectory}. 
  */
 public final class DirectoryWithQuotaFeature implements INode.Feature {
   public static final long DEFAULT_NAMESPACE_QUOTA = Long.MAX_VALUE;
-  public static final long DEFAULT_DISKSPACE_QUOTA = HdfsConstants.QUOTA_RESET;
-
-  /** Name space quota */
-  private long nsQuota = DEFAULT_NAMESPACE_QUOTA;
-  /** Name space count */
-  private long namespace = 1L;
-  /** Disk space quota */
-  private long dsQuota = DEFAULT_DISKSPACE_QUOTA;
-  /** Disk space count */
-  private long diskspace = 0L;
-  
-  DirectoryWithQuotaFeature(long nsQuota, long dsQuota) {
-    this.nsQuota = nsQuota;
-    this.dsQuota = dsQuota;
+  public static final long DEFAULT_SPACE_QUOTA = HdfsConstants.QUOTA_RESET;
+
+  private QuotaCounts quota;
+  private QuotaCounts usage;
+
+  public static class Builder {
+    private QuotaCounts quota;
+    private QuotaCounts usage;
+
+    public Builder() {
+      this.quota = new QuotaCounts.Builder().nameCount(DEFAULT_NAMESPACE_QUOTA).
+          spaceCount(DEFAULT_SPACE_QUOTA).typeCounts(DEFAULT_SPACE_QUOTA).build();
+      this.usage = new QuotaCounts.Builder().nameCount(1).build();
+    }
+
+    public Builder nameSpaceQuota(long nameSpaceQuota) {
+      this.quota.setNameSpace(nameSpaceQuota);
+      return this;
+    }
+
+    public Builder spaceQuota(long spaceQuota) {
+      this.quota.setDiskSpace(spaceQuota);
+      return this;
+    }
+
+    public Builder typeQuotas(EnumCounters<StorageType> typeQuotas) {
+      this.quota.setTypeSpaces(typeQuotas);
+      return this;
+    }
+
+    public Builder typeQuota(StorageType type, long quota) {
+      this.quota.setTypeSpace(type, quota);
+      return this;
+    }
+
+    public DirectoryWithQuotaFeature build() {
+      return new DirectoryWithQuotaFeature(this);
+    }
+  }
+
+  private DirectoryWithQuotaFeature(Builder builder) {
+    this.quota = builder.quota;
+    this.usage = builder.usage;
   }
 
   /** @return the quota set or -1 if it is not set. */
-  Quota.Counts getQuota() {
-    return Quota.Counts.newInstance(nsQuota, dsQuota);
+  QuotaCounts getQuota() {
+    return new QuotaCounts.Builder().quotaCount(this.quota).build();
   }
-  
+
   /** Set this directory's quota
    * 
    * @param nsQuota Namespace quota to be set
    * @param dsQuota Diskspace quota to be set
+   * @param type Storage type quota to be set
+   * * To set traditional space/namespace quota, type must be null
    */
+  void setQuota(long nsQuota, long dsQuota, StorageType type) {
+    if (type != null) {
+      this.quota.setTypeSpace(type, dsQuota);
+    } else {
+      setQuota(nsQuota, dsQuota);
+    }
+  }
+
   void setQuota(long nsQuota, long dsQuota) {
-    this.nsQuota = nsQuota;
-    this.dsQuota = dsQuota;
+    this.quota.setNameSpace(nsQuota);
+    this.quota.setDiskSpace(dsQuota);
   }
-  
-  Quota.Counts addNamespaceDiskspace(Quota.Counts counts) {
-    counts.add(Quota.NAMESPACE, namespace);
-    counts.add(Quota.DISKSPACE, diskspace);
+
+  void setQuota(long dsQuota, StorageType type) {
+    this.quota.setTypeSpace(type, dsQuota);
+  }
+
+  // Set in a batch only during FSImage load
+  void setQuota(EnumCounters<StorageType> typeQuotas) {
+    this.quota.setTypeSpaces(typeQuotas);
+  }
+
+  QuotaCounts addNamespaceDiskspace(QuotaCounts counts) {
+    counts.add(this.usage);
     return counts;
   }
 
@@ -76,45 +126,42 @@ public final class DirectoryWithQuotaFeature implements INode.Feature {
     }
     return summary;
   }
-  
+
   private void checkDiskspace(final INodeDirectory dir, final long computed) {
-    if (-1 != getQuota().get(Quota.DISKSPACE) && diskspace != computed) {
+    if (-1 != quota.getDiskSpace() && usage.getDiskSpace() != computed) {
       NameNode.LOG.error("BUG: Inconsistent diskspace for directory "
-          + dir.getFullPathName() + ". Cached = " + diskspace
+          + dir.getFullPathName() + ". Cached = " + usage.getDiskSpace()
           + " != Computed = " + computed);
     }
   }
 
-  void addSpaceConsumed(final INodeDirectory dir, final long nsDelta,
-      final long dsDelta, boolean verify) throws QuotaExceededException {
-    if (dir.isQuotaSet()) { 
-      // The following steps are important: 
+  void addSpaceConsumed(final INodeDirectory dir, final QuotaCounts counts,
+      boolean verify) throws QuotaExceededException {
+    if (dir.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
       if (verify) {
-        verifyQuota(nsDelta, dsDelta);
+        verifyQuota(counts);
       }
-      // (2) verify quota and then add count in ancestors 
-      dir.addSpaceConsumed2Parent(nsDelta, dsDelta, verify);
+      // (2) verify quota and then add count in ancestors
+      dir.addSpaceConsumed2Parent(counts, verify);
       // (3) add count in this inode
-      addSpaceConsumed2Cache(nsDelta, dsDelta);
+      addSpaceConsumed2Cache(counts);
     } else {
-      dir.addSpaceConsumed2Parent(nsDelta, dsDelta, verify);
+      dir.addSpaceConsumed2Parent(counts, verify);
     }
   }
   
-  /** Update the size of the tree
+  /** Update the space/namespace/type usage of the tree
    * 
-   * @param nsDelta the change of the tree size
-   * @param dsDelta change to disk space occupied
+   * @param delta the change of the namespace/space/type usage
    */
-  public void addSpaceConsumed2Cache(long nsDelta, long dsDelta) {
-    namespace += nsDelta;
-    diskspace += dsDelta;
+  public void addSpaceConsumed2Cache(QuotaCounts delta) {
+    usage.add(delta);
   }
-  
+
   /** 
    * Sets namespace and diskspace take by the directory rooted 
    * at this INode. This should be used carefully. It does not check 
@@ -122,52 +169,103 @@ public final class DirectoryWithQuotaFeature implements INode.Feature {
    * 
    * @param namespace size of the directory to be set
    * @param diskspace disk space take by all the nodes under this directory
+   * @param typeUsed counters of storage type usage
    */
-  void setSpaceConsumed(long namespace, long diskspace) {
-    this.namespace = namespace;
-    this.diskspace = diskspace;
+  void setSpaceConsumed(long namespace, long diskspace,
+      EnumCounters<StorageType> typeUsed) {
+    usage.setNameSpace(namespace);
+    usage.setDiskSpace(diskspace);
+    usage.setTypeSpaces(typeUsed);
   }
-  
+
+  void setSpaceConsumed(QuotaCounts c) {
+    usage.setNameSpace(c.getNameSpace());
+    usage.setDiskSpace(c.getDiskSpace());
+    usage.setTypeSpaces(c.getTypeSpaces());
+  }
+
   /** @return the namespace and diskspace consumed. */
-  public Quota.Counts getSpaceConsumed() {
-    return Quota.Counts.newInstance(namespace, diskspace);
+  public QuotaCounts getSpaceConsumed() {
+    return new QuotaCounts.Builder().quotaCount(usage).build();
   }
 
   /** Verify if the namespace quota is violated after applying delta. */
   private void verifyNamespaceQuota(long delta) throws NSQuotaExceededException {
-    if (Quota.isViolated(nsQuota, namespace, delta)) {
-      throw new NSQuotaExceededException(nsQuota, namespace + delta);
+    if (Quota.isViolated(quota.getNameSpace(), usage.getNameSpace(), delta)) {
+      throw new NSQuotaExceededException(quota.getNameSpace(),
+          usage.getNameSpace() + delta);
     }
   }
   /** Verify if the diskspace quota is violated after applying delta. */
   private void verifyDiskspaceQuota(long delta) throws DSQuotaExceededException {
-    if (Quota.isViolated(dsQuota, diskspace, delta)) {
-      throw new DSQuotaExceededException(dsQuota, diskspace + delta);
+    if (Quota.isViolated(quota.getDiskSpace(), usage.getDiskSpace(), delta)) {
+      throw new DSQuotaExceededException(quota.getDiskSpace(),
+          usage.getDiskSpace() + delta);
+    }
+  }
+
+  private void verifyQuotaByStorageType(EnumCounters<StorageType> typeDelta)
+      throws QuotaByStorageTypeExceededException {
+    if (!isQuotaByStorageTypeSet()) {
+      return;
+    }
+    for (StorageType t: StorageType.getTypesSupportingQuota()) {
+      if (!isQuotaByStorageTypeSet(t)) {
+        continue;
+      }
+      if (Quota.isViolated(quota.getTypeSpace(t), usage.getTypeSpace(t),
+          typeDelta.get(t))) {
+        throw new QuotaByStorageTypeExceededException(
+          quota.getTypeSpace(t), usage.getTypeSpace(t) + typeDelta.get(t), t);
+      }
     }
   }
 
   /**
-   * @throws QuotaExceededException if namespace or diskspace quotas is
-   *         violated after applying the deltas.
+   * @throws QuotaExceededException if namespace, diskspace or storage type quotas
+   * is violated after applying the deltas.
    */
-  void verifyQuota(long nsDelta, long dsDelta) throws QuotaExceededException {
-    verifyNamespaceQuota(nsDelta);
-    verifyDiskspaceQuota(dsDelta);
+  void verifyQuota(QuotaCounts counts) throws QuotaExceededException {
+    verifyNamespaceQuota(counts.getNameSpace());
+    verifyDiskspaceQuota(counts.getDiskSpace());
+    verifyQuotaByStorageType(counts.getTypeSpaces());
   }
-  
+
   boolean isQuotaSet() {
-    return nsQuota >= 0 || dsQuota >= 0;
+    return quota.anyNsSpCountGreaterOrEqual(0) ||
+        quota.anyTypeCountGreaterOrEqual(0);
+  }
+
+  boolean isQuotaByStorageTypeSet() {
+    return quota.anyTypeCountGreaterOrEqual(0);
+  }
+
+  boolean isQuotaByStorageTypeSet(StorageType t) {
+    return quota.getTypeSpace(t) >= 0;
   }
 
   private String namespaceString() {
-    return "namespace: " + (nsQuota < 0? "-": namespace + "/" + nsQuota);
+    return "namespace: " + (quota.getNameSpace() < 0? "-":
+        usage.getNameSpace() + "/" + quota.getNameSpace());
   }
   private String diskspaceString() {
-    return "diskspace: " + (dsQuota < 0? "-": diskspace + "/" + dsQuota);
+    return "diskspace: " + (quota.getDiskSpace() < 0? "-":
+        usage.getDiskSpace() + "/" + quota.getDiskSpace());
   }
-  
+
+  private String quotaByStorageTypeString() {
+    StringBuilder sb = new StringBuilder();
+    for (StorageType t : StorageType.getTypesSupportingQuota()) {
+      sb.append("StorageType: " + t +
+          (quota.getTypeSpace(t) < 0? "-":
+          usage.getTypeSpace(t) + "/" + usage.getTypeSpace(t)));
+    }
+    return sb.toString();
+  }
+
   @Override
   public String toString() {
-    return "Quota[" + namespaceString() + ", " + diskspaceString() + "]";
+    return "Quota[" + namespaceString() + ", " + diskspaceString() +
+        ", " + quotaByStorageTypeString() + "]";
   }
-}
+}

+ 46 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java

@@ -32,6 +32,8 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.security.AccessControlException;
 
 import java.io.FileNotFoundException;
@@ -41,6 +43,7 @@ import java.util.EnumSet;
 import java.util.List;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_QUOTA_BY_STORAGETYPE_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
 
 public class FSDirAttrOp {
@@ -216,11 +219,11 @@ public class FSDirAttrOp {
   }
 
   /**
-   * Set the namespace quota and diskspace quota for a directory.
+   * Set the namespace quota, diskspace and typeSpace quota for a directory.
    *
    * Note: This does not support ".inodes" relative path.
    */
-  static void setQuota(FSDirectory fsd, String src, long nsQuota, long dsQuota)
+  static void setQuota(FSDirectory fsd, String src, long nsQuota, long dsQuota, StorageType type)
       throws IOException {
     if (fsd.isPermissionEnabled()) {
       FSPermissionChecker pc = fsd.getPermissionChecker();
@@ -229,11 +232,15 @@ public class FSDirAttrOp {
 
     fsd.writeLock();
     try {
-      INodeDirectory changed = unprotectedSetQuota(fsd, src, nsQuota, dsQuota);
+      INodeDirectory changed = unprotectedSetQuota(fsd, src, nsQuota, dsQuota, type);
       if (changed != null) {
-        final Quota.Counts q = changed.getQuotaCounts();
-        fsd.getEditLog().logSetQuota(
-            src, q.get(Quota.NAMESPACE), q.get(Quota.DISKSPACE));
+        final QuotaCounts q = changed.getQuotaCounts();
+        if (type == null) {
+          fsd.getEditLog().logSetQuota(src, q.getNameSpace(), q.getDiskSpace());
+        } else {
+          fsd.getEditLog().logSetQuotaByStorageType(
+              src, q.getTypeSpaces().get(type), type);
+        }
       }
     } finally {
       fsd.writeUnlock();
@@ -294,7 +301,8 @@ public class FSDirAttrOp {
   }
 
   /**
-   * See {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long)}
+   * See {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String,
+   *     long, long, StorageType)}
    * for the contract.
    * Sets quota for for a directory.
    * @return INodeDirectory if any of the quotas have changed. null otherwise.
@@ -306,10 +314,10 @@ public class FSDirAttrOp {
    * @throws SnapshotAccessControlException if path is in RO snapshot
    */
   static INodeDirectory unprotectedSetQuota(
-      FSDirectory fsd, String src, long nsQuota, long dsQuota)
+      FSDirectory fsd, String src, long nsQuota, long dsQuota, StorageType type)
       throws FileNotFoundException, PathIsNotDirectoryException,
       QuotaExceededException, UnresolvedLinkException,
-      SnapshotAccessControlException {
+      SnapshotAccessControlException, UnsupportedActionException {
     assert fsd.hasWriteLock();
     // sanity check
     if ((nsQuota < 0 && nsQuota != HdfsConstants.QUOTA_DONT_SET &&
@@ -320,6 +328,15 @@ public class FSDirAttrOp {
                                          "dsQuota : " + nsQuota + " and " +
                                          dsQuota);
     }
+    // sanity check for quota by storage type
+    if ((type != null) && (!fsd.isQuotaByStorageTypeEnabled() ||
+        nsQuota != HdfsConstants.QUOTA_DONT_SET)) {
+      throw new UnsupportedActionException(
+          "Failed to set quota by storage type because either" +
+          DFS_QUOTA_BY_STORAGETYPE_ENABLED_KEY + " is set to " +
+          fsd.isQuotaByStorageTypeEnabled() + " or nsQuota value is illegal " +
+          nsQuota);
+    }
 
     String srcs = FSDirectory.normalizePath(src);
     final INodesInPath iip = fsd.getINodesInPath4Write(srcs, true);
@@ -327,22 +344,33 @@ public class FSDirAttrOp {
     if (dirNode.isRoot() && nsQuota == HdfsConstants.QUOTA_RESET) {
       throw new IllegalArgumentException("Cannot clear namespace quota on root.");
     } else { // a directory inode
-      final Quota.Counts oldQuota = dirNode.getQuotaCounts();
-      final long oldNsQuota = oldQuota.get(Quota.NAMESPACE);
-      final long oldDsQuota = oldQuota.get(Quota.DISKSPACE);
+      final QuotaCounts oldQuota = dirNode.getQuotaCounts();
+      final long oldNsQuota = oldQuota.getNameSpace();
+      final long oldDsQuota = oldQuota.getDiskSpace();
+
       if (nsQuota == HdfsConstants.QUOTA_DONT_SET) {
         nsQuota = oldNsQuota;
       }
       if (dsQuota == HdfsConstants.QUOTA_DONT_SET) {
         dsQuota = oldDsQuota;
       }
-      if (oldNsQuota == nsQuota && oldDsQuota == dsQuota) {
+
+      // unchanged space/namespace quota
+      if (type == null && oldNsQuota == nsQuota && oldDsQuota == dsQuota) {
         return null;
       }
 
+      // unchanged type quota
+      if (type != null) {
+          EnumCounters<StorageType> oldTypeQuotas = oldQuota.getTypeSpaces();
+          if (oldTypeQuotas != null && oldTypeQuotas.get(type) == dsQuota) {
+              return null;
+          }
+      }
+
       final int latest = iip.getLatestSnapshotId();
       dirNode.recordModification(latest);
-      dirNode.setQuota(nsQuota, dsQuota);
+      dirNode.setQuota(fsd.getBlockStoragePolicySuite(), nsQuota, dsQuota, type);
       return dirNode;
     }
   }
@@ -365,8 +393,8 @@ public class FSDirAttrOp {
     // if replication > oldBR, then newBR == replication.
     // if replication < oldBR, we don't know newBR yet.
     if (replication > oldBR) {
-      long dsDelta = (replication - oldBR)*(file.diskspaceConsumed()/oldBR);
-      fsd.updateCount(iip, 0, dsDelta, true);
+      long dsDelta = file.diskspaceConsumed()/oldBR;
+      fsd.updateCount(iip, 0L, dsDelta, oldBR, replication, true);
     }
 
     file.setFileReplication(replication, iip.getLatestSnapshotId());
@@ -374,8 +402,8 @@ public class FSDirAttrOp {
     final short newBR = file.getBlockReplication();
     // check newBR < oldBR case.
     if (newBR < oldBR) {
-      long dsDelta = (newBR - oldBR)*(file.diskspaceConsumed()/newBR);
-      fsd.updateCount(iip, 0, dsDelta, true);
+      long dsDelta = file.diskspaceConsumed()/newBR;
+      fsd.updateCount(iip, 0L, dsDelta, oldBR, newBR, true);
     }
 
     if (blockRepls != null) {

+ 32 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java

@@ -20,14 +20,17 @@ package org.apache.hadoop.hdfs.server.namenode;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
+import org.apache.hadoop.hdfs.StorageType;
 
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.List;
 
 import static org.apache.hadoop.util.Time.now;
 
@@ -140,25 +143,42 @@ class FSDirConcatOp {
     return si.toArray(new INodeFile[si.size()]);
   }
 
-  private static long computeQuotaDelta(INodeFile target, INodeFile[] srcList) {
-    long delta = 0;
+  private static QuotaCounts computeQuotaDeltas(FSDirectory fsd, INodeFile target, INodeFile[] srcList) {
+    QuotaCounts deltas = new QuotaCounts.Builder().build();
     short targetRepl = target.getBlockReplication();
     for (INodeFile src : srcList) {
-      if (targetRepl != src.getBlockReplication()) {
-        delta += src.computeFileSize() *
-            (targetRepl - src.getBlockReplication());
+      short srcRepl = src.getBlockReplication();
+      long fileSize = src.computeFileSize();
+      if (targetRepl != srcRepl) {
+        deltas.addDiskSpace(fileSize * (targetRepl - srcRepl));
+        BlockStoragePolicy bsp =
+            fsd.getBlockStoragePolicySuite().getPolicy(src.getStoragePolicyID());
+        if (bsp != null) {
+          List<StorageType> srcTypeChosen = bsp.chooseStorageTypes(srcRepl);
+          for (StorageType t : srcTypeChosen) {
+            if (t.supportTypeQuota()) {
+              deltas.addTypeSpace(t, -fileSize);
+            }
+          }
+          List<StorageType> targetTypeChosen = bsp.chooseStorageTypes(targetRepl);
+          for (StorageType t : targetTypeChosen) {
+            if (t.supportTypeQuota()) {
+              deltas.addTypeSpace(t, fileSize);
+            }
+          }
+        }
       }
     }
-    return delta;
+    return deltas;
   }
 
   private static void verifyQuota(FSDirectory fsd, INodesInPath targetIIP,
-      long delta) throws QuotaExceededException {
+      QuotaCounts deltas) throws QuotaExceededException {
     if (!fsd.getFSNamesystem().isImageLoaded() || fsd.shouldSkipQuotaChecks()) {
       // Do not check quota if editlog is still being processed
       return;
     }
-    FSDirectory.verifyQuota(targetIIP, targetIIP.length() - 1, 0, delta, null);
+    FSDirectory.verifyQuota(targetIIP, targetIIP.length() - 1, deltas, null);
   }
 
   /**
@@ -174,8 +194,8 @@ class FSDirConcatOp {
     }
 
     final INodeFile trgInode = targetIIP.getLastINode().asFile();
-    long delta = computeQuotaDelta(trgInode, srcList);
-    verifyQuota(fsd, targetIIP, delta);
+    QuotaCounts deltas = computeQuotaDeltas(fsd, trgInode, srcList);
+    verifyQuota(fsd, targetIIP, deltas);
 
     // the target file can be included in a snapshot
     trgInode.recordModification(targetIIP.getLatestSnapshotId());
@@ -195,8 +215,7 @@ class FSDirConcatOp {
 
     trgInode.setModificationTime(timestamp, targetIIP.getLatestSnapshotId());
     trgParent.updateModificationTime(timestamp, targetIIP.getLatestSnapshotId());
-    // update quota on the parent directory ('count' files removed, 0 space)
-    FSDirectory.unprotectedUpdateCount(targetIIP, targetIIP.length() - 1,
-        -count, delta);
+    // update quota on the parent directory with deltas
+    FSDirectory.unprotectedUpdateCount(targetIIP, targetIIP.length() - 1, deltas);
   }
 }

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

@@ -230,13 +230,14 @@ class FSDirDeleteOp {
 
     // collect block and update quota
     if (!targetNode.isInLatestSnapshot(latestSnapshot)) {
-      targetNode.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+      targetNode.destroyAndCollectBlocks(fsd.getBlockStoragePolicySuite(),
+        collectedBlocks, removedINodes);
     } else {
-      Quota.Counts counts = targetNode.cleanSubtree(CURRENT_STATE_ID,
+      QuotaCounts counts = targetNode.cleanSubtree(
+        fsd.getBlockStoragePolicySuite(), CURRENT_STATE_ID,
           latestSnapshot, collectedBlocks, removedINodes);
-      removed = counts.get(Quota.NAMESPACE);
-      fsd.updateCountNoQuotaCheck(iip, iip.length() - 1,
-          -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
+      removed = counts.getNameSpace();
+      fsd.updateCountNoQuotaCheck(iip, iip.length() -1, counts.negation());
     }
 
     if (NameNode.stateChangeLog.isDebugEnabled()) {

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

@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -90,16 +91,15 @@ class FSDirRenameOp {
     int i = 0;
     while(src.getINode(i) == dst.getINode(i)) { i++; }
     // src[i - 1] is the last common ancestor.
-
-    final Quota.Counts delta = src.getLastINode().computeQuotaUsage();
+    BlockStoragePolicySuite bsps = fsd.getBlockStoragePolicySuite();
+    final QuotaCounts delta = src.getLastINode().computeQuotaUsage(bsps);
 
     // Reduce the required quota by dst that is being removed
     final INode dstINode = dst.getLastINode();
     if (dstINode != null) {
-      delta.subtract(dstINode.computeQuotaUsage());
+      delta.subtract(dstINode.computeQuotaUsage(bsps));
     }
-    FSDirectory.verifyQuota(dst, dst.length() - 1, delta.get(Quota.NAMESPACE),
-        delta.get(Quota.DISKSPACE), src.getINode(i - 1));
+    FSDirectory.verifyQuota(dst, dst.length() - 1, delta, src.getINode(i - 1));
   }
 
   /**
@@ -207,7 +207,7 @@ class FSDirRenameOp {
         }
 
         tx.updateMtimeAndLease(timestamp);
-        tx.updateQuotasInSourceTree();
+        tx.updateQuotasInSourceTree(fsd.getBlockStoragePolicySuite());
 
         return true;
       }
@@ -356,6 +356,7 @@ class FSDirRenameOp {
       throw new IOException(error);
     }
 
+    BlockStoragePolicySuite bsps = fsd.getBlockStoragePolicySuite();
     fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src);
     final INode dstInode = dstIIP.getLastINode();
     List<INodeDirectory> snapshottableDirs = new ArrayList<>();
@@ -412,7 +413,7 @@ class FSDirRenameOp {
         if (undoRemoveDst) {
           undoRemoveDst = false;
           if (removedNum > 0) {
-            filesDeleted = tx.cleanDst(collectedBlocks);
+            filesDeleted = tx.cleanDst(bsps, collectedBlocks);
           }
         }
 
@@ -422,7 +423,7 @@ class FSDirRenameOp {
           fsd.getFSNamesystem().removeSnapshottableDirs(snapshottableDirs);
         }
 
-        tx.updateQuotasInSourceTree();
+        tx.updateQuotasInSourceTree(bsps);
         return filesDeleted;
       }
     } finally {
@@ -430,7 +431,7 @@ class FSDirRenameOp {
         tx.restoreSource();
       }
       if (undoRemoveDst) { // Rename failed - restore dst
-        tx.restoreDst();
+        tx.restoreDst(bsps);
       }
     }
     NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: " +
@@ -566,7 +567,7 @@ class FSDirRenameOp {
     private final byte[] srcChildName;
     private final boolean isSrcInSnapshot;
     private final boolean srcChildIsReference;
-    private final Quota.Counts oldSrcCounts;
+    private final QuotaCounts oldSrcCounts;
     private INode srcChild;
     private INode oldDstChild;
 
@@ -581,6 +582,7 @@ class FSDirRenameOp {
       this.srcParentIIP = srcIIP.getParentINodesInPath();
       this.dstParentIIP = dstIIP.getParentINodesInPath();
 
+      BlockStoragePolicySuite bsps = fsd.getBlockStoragePolicySuite();
       srcChild = this.srcIIP.getLastINode();
       srcChildName = srcChild.getLocalNameBytes();
       final int srcLatestSnapshotId = srcIIP.getLatestSnapshotId();
@@ -598,7 +600,7 @@ class FSDirRenameOp {
       // check srcChild for reference
       srcRefDstSnapshot = srcChildIsReference ?
           srcChild.asReference().getDstSnapshotId() : Snapshot.CURRENT_STATE_ID;
-      oldSrcCounts = Quota.Counts.newInstance();
+      oldSrcCounts = new QuotaCounts.Builder().build();
       if (isSrcInSnapshot) {
         final INodeReference.WithName withName = srcParent
             .replaceChild4ReferenceWithName(srcChild, srcLatestSnapshotId);
@@ -607,7 +609,7 @@ class FSDirRenameOp {
         this.srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1,
             srcChild);
         // get the counts before rename
-        withCount.getReferredINode().computeQuotaUsage(oldSrcCounts, true);
+        withCount.getReferredINode().computeQuotaUsage(bsps, oldSrcCounts, true);
       } else if (srcChildIsReference) {
         // srcChild is reference but srcChild is not in latest snapshot
         withCount = (INodeReference.WithCount) srcChild.asReference()
@@ -709,11 +711,11 @@ class FSDirRenameOp {
       }
     }
 
-    void restoreDst() throws QuotaExceededException {
+    void restoreDst(BlockStoragePolicySuite bsps) throws QuotaExceededException {
       Preconditions.checkState(oldDstChild != null);
       final INodeDirectory dstParent = dstParentIIP.getLastINode().asDirectory();
       if (dstParent.isWithSnapshot()) {
-        dstParent.undoRename4DstParent(oldDstChild, dstIIP.getLatestSnapshotId());
+        dstParent.undoRename4DstParent(bsps, oldDstChild, dstIIP.getLatestSnapshotId());
       } else {
         fsd.addLastINodeNoQuotaCheck(dstParentIIP, oldDstChild);
       }
@@ -725,32 +727,31 @@ class FSDirRenameOp {
       }
     }
 
-    boolean cleanDst(BlocksMapUpdateInfo collectedBlocks)
+    boolean cleanDst(BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks)
         throws QuotaExceededException {
       Preconditions.checkState(oldDstChild != null);
       List<INode> removedINodes = new ChunkedArrayList<>();
       final boolean filesDeleted;
       if (!oldDstChild.isInLatestSnapshot(dstIIP.getLatestSnapshotId())) {
-        oldDstChild.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+        oldDstChild.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes);
         filesDeleted = true;
       } else {
-        filesDeleted = oldDstChild.cleanSubtree(Snapshot.CURRENT_STATE_ID,
+        filesDeleted = oldDstChild.cleanSubtree(bsps, Snapshot.CURRENT_STATE_ID,
             dstIIP.getLatestSnapshotId(), collectedBlocks, removedINodes)
-            .get(Quota.NAMESPACE) >= 0;
+            .getNameSpace() >= 0;
       }
       fsd.getFSNamesystem().removeLeasesAndINodes(src, removedINodes, false);
       return filesDeleted;
     }
 
-    void updateQuotasInSourceTree() throws QuotaExceededException {
+    void updateQuotasInSourceTree(BlockStoragePolicySuite bsps) throws QuotaExceededException {
       // update the quota usage in src tree
       if (isSrcInSnapshot) {
         // get the counts after rename
-        Quota.Counts newSrcCounts = srcChild.computeQuotaUsage(
-            Quota.Counts.newInstance(), false);
+        QuotaCounts newSrcCounts = srcChild.computeQuotaUsage(bsps,
+            new QuotaCounts.Builder().build(), false);
         newSrcCounts.subtract(oldSrcCounts);
-        srcParent.addSpaceConsumed(newSrcCounts.get(Quota.NAMESPACE),
-            newSrcCounts.get(Quota.DISKSPACE), false);
+        srcParent.addSpaceConsumed(newSrcCounts, false);
       }
     }
   }

+ 125 - 46
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.AclException;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
@@ -56,9 +57,12 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.util.ByteArray;
+import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
@@ -76,6 +80,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_QUOTA_BY_STORAGETYPE_ENABLED_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_QUOTA_BY_STORAGETYPE_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_ENCRYPTION_ZONE;
@@ -100,8 +106,9 @@ public class FSDirectory implements Closeable {
         namesystem.createFsOwnerPermissions(new FsPermission((short) 0755)),
         0L);
     r.addDirectoryWithQuotaFeature(
-        DirectoryWithQuotaFeature.DEFAULT_NAMESPACE_QUOTA,
-        DirectoryWithQuotaFeature.DEFAULT_DISKSPACE_QUOTA);
+        new DirectoryWithQuotaFeature.Builder().
+            nameSpaceQuota(DirectoryWithQuotaFeature.DEFAULT_NAMESPACE_QUOTA).
+            spaceQuota(DirectoryWithQuotaFeature.DEFAULT_SPACE_QUOTA).build());
     r.addSnapshottableFeature();
     r.setSnapshotQuota(0);
     return r;
@@ -149,6 +156,8 @@ public class FSDirectory implements Closeable {
   private final long accessTimePrecision;
   // whether setStoragePolicy is allowed.
   private final boolean storagePolicyEnabled;
+  // whether quota by storage type is allowed
+  private final boolean quotaByStorageTypeEnabled;
 
   private final String fsOwnerShortUserName;
   private final String supergroup;
@@ -236,6 +245,10 @@ public class FSDirectory implements Closeable {
         conf.getBoolean(DFS_STORAGE_POLICY_ENABLED_KEY,
                         DFS_STORAGE_POLICY_ENABLED_DEFAULT);
 
+    this.quotaByStorageTypeEnabled =
+        conf.getBoolean(DFS_QUOTA_BY_STORAGETYPE_ENABLED_KEY,
+                        DFS_QUOTA_BY_STORAGETYPE_ENABLED_DEFAULT);
+
     int configuredLimit = conf.getInt(
         DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
     this.lsLimit = configuredLimit>0 ?
@@ -291,6 +304,10 @@ public class FSDirectory implements Closeable {
     return rootDir;
   }
 
+  public BlockStoragePolicySuite getBlockStoragePolicySuite() {
+    return getBlockManager().getStoragePolicySuite();
+  }
+
   boolean isPermissionEnabled() {
     return isPermissionEnabled;
   }
@@ -307,6 +324,9 @@ public class FSDirectory implements Closeable {
   boolean isAccessTimeSupported() {
     return accessTimePrecision > 0;
   }
+  boolean isQuotaByStorageTypeEnabled() {
+    return quotaByStorageTypeEnabled;
+  }
 
 
   int getLsLimit() {
@@ -452,7 +472,8 @@ public class FSDirectory implements Closeable {
       Preconditions.checkState(fileINode.isUnderConstruction());
 
       // check quota limits and updated space consumed
-      updateCount(inodesInPath, 0, fileINode.getPreferredBlockDiskspace(), true);
+      updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
+          fileINode.getBlockReplication(), true);
 
       // associate new last block for the file
       BlockInfoContiguousUnderConstruction blockInfo =
@@ -508,7 +529,8 @@ public class FSDirectory implements Closeable {
     }
 
     // update space consumed
-    updateCount(iip, 0, -fileNode.getPreferredBlockDiskspace(), true);
+    updateCount(iip, 0, -fileNode.getPreferredBlockSize(),
+        fileNode.getBlockReplication(), true);
     return true;
   }
 
@@ -584,19 +606,20 @@ public class FSDirectory implements Closeable {
    * @param iip the INodesInPath instance containing all the INodes for
    *            updating quota usage
    * @param nsDelta the delta change of namespace
-   * @param dsDelta the delta change of diskspace
+   * @param dsDelta the delta change of space consumed without replication
+   * @param replication the replication factor of the block consumption change
    * @throws QuotaExceededException if the new count violates any quota limit
    * @throws FileNotFoundException if path does not exist.
    */
-  void updateSpaceConsumed(INodesInPath iip, long nsDelta, long dsDelta)
-      throws QuotaExceededException, FileNotFoundException,
-          UnresolvedLinkException, SnapshotAccessControlException {
+  void updateSpaceConsumed(INodesInPath iip, long nsDelta, long dsDelta, short replication)
+    throws QuotaExceededException, FileNotFoundException,
+    UnresolvedLinkException, SnapshotAccessControlException {
     writeLock();
     try {
       if (iip.getLastINode() == null) {
         throw new FileNotFoundException("Path not found: " + iip.getPath());
       }
-      updateCount(iip, nsDelta, dsDelta, true);
+      updateCount(iip, nsDelta, dsDelta, replication, true);
     } finally {
       writeUnlock();
     }
@@ -610,29 +633,52 @@ public class FSDirectory implements Closeable {
   void updateCountForDelete(final INode inode, final INodesInPath iip) {
     if (getFSNamesystem().isImageLoaded() &&
         !inode.isInLatestSnapshot(iip.getLatestSnapshotId())) {
-      Quota.Counts counts = inode.computeQuotaUsage();
-      unprotectedUpdateCount(iip, iip.length() - 1,
-          -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
+      QuotaCounts counts = inode.computeQuotaUsage(getBlockStoragePolicySuite());
+      unprotectedUpdateCount(iip, iip.length() - 1, counts.negation());
     }
   }
 
-  void updateCount(INodesInPath iip, long nsDelta, long dsDelta,
+  /**
+   * Update usage count without replication factor change
+   */
+  void updateCount(INodesInPath iip, long nsDelta, long dsDelta, short replication,
       boolean checkQuota) throws QuotaExceededException {
-    updateCount(iip, iip.length() - 1, nsDelta, dsDelta, checkQuota);
+    final INodeFile fileINode = iip.getLastINode().asFile();
+    EnumCounters<StorageType> typeSpaceDeltas =
+      getStorageTypeDeltas(fileINode.getStoragePolicyID(), dsDelta,
+          replication, replication);;
+    updateCount(iip, iip.length() - 1,
+      new QuotaCounts.Builder().nameCount(nsDelta).spaceCount(dsDelta * replication).
+          typeCounts(typeSpaceDeltas).build(),
+        checkQuota);
+  }
+
+  /**
+   * Update usage count with replication factor change due to setReplication
+   */
+  void updateCount(INodesInPath iip, long nsDelta, long dsDelta, short oldRep,
+      short newRep, boolean checkQuota) throws QuotaExceededException {
+    final INodeFile fileINode = iip.getLastINode().asFile();
+    EnumCounters<StorageType> typeSpaceDeltas =
+        getStorageTypeDeltas(fileINode.getStoragePolicyID(), dsDelta, oldRep, newRep);
+    updateCount(iip, iip.length() - 1,
+        new QuotaCounts.Builder().nameCount(nsDelta).
+            spaceCount(dsDelta * (newRep - oldRep)).
+            typeCounts(typeSpaceDeltas).build(),
+        checkQuota);
   }
 
   /** update count of each inode with quota
    * 
    * @param iip inodes in a path
    * @param numOfINodes the number of inodes to update starting from index 0
-   * @param nsDelta the delta change of namespace
-   * @param dsDelta the delta change of diskspace
+   * @param counts the count of space/namespace/type usage to be update
    * @param checkQuota if true then check if quota is exceeded
    * @throws QuotaExceededException if the new count violates any quota limit
    */
-  private void updateCount(INodesInPath iip, int numOfINodes, 
-                           long nsDelta, long dsDelta, boolean checkQuota)
-                           throws QuotaExceededException {
+   void updateCount(INodesInPath iip, int numOfINodes,
+                    QuotaCounts counts, boolean checkQuota)
+                    throws QuotaExceededException {
     assert hasWriteLock();
     if (!namesystem.isImageLoaded()) {
       //still initializing. do not check or update quotas.
@@ -642,20 +688,20 @@ public class FSDirectory implements Closeable {
       numOfINodes = iip.length();
     }
     if (checkQuota && !skipQuotaCheck) {
-      verifyQuota(iip, numOfINodes, nsDelta, dsDelta, null);
+      verifyQuota(iip, numOfINodes, counts, null);
     }
-    unprotectedUpdateCount(iip, numOfINodes, nsDelta, dsDelta);
+    unprotectedUpdateCount(iip, numOfINodes, counts);
   }
   
   /** 
    * update quota of each inode and check to see if quota is exceeded. 
-   * See {@link #updateCount(INodesInPath, long, long, boolean)}
+   * See {@link #updateCount(INodesInPath, int, QuotaCounts, boolean)}
    */ 
-  void updateCountNoQuotaCheck(INodesInPath inodesInPath, int numOfINodes,
-      long nsDelta, long dsDelta) {
+   void updateCountNoQuotaCheck(INodesInPath inodesInPath,
+      int numOfINodes, QuotaCounts counts) {
     assert hasWriteLock();
     try {
-      updateCount(inodesInPath, numOfINodes, nsDelta, dsDelta, false);
+      updateCount(inodesInPath, numOfINodes, counts, false);
     } catch (QuotaExceededException e) {
       NameNode.LOG.error("BUG: unexpected exception ", e);
     }
@@ -666,15 +712,49 @@ public class FSDirectory implements Closeable {
    * callers responsibility is to make sure quota is not exceeded
    */
   static void unprotectedUpdateCount(INodesInPath inodesInPath,
-      int numOfINodes, long nsDelta, long dsDelta) {
+      int numOfINodes, QuotaCounts counts) {
     for(int i=0; i < numOfINodes; i++) {
       if (inodesInPath.getINode(i).isQuotaSet()) { // a directory with quota
         inodesInPath.getINode(i).asDirectory().getDirectoryWithQuotaFeature()
-            .addSpaceConsumed2Cache(nsDelta, dsDelta);
+            .addSpaceConsumed2Cache(counts);
       }
     }
   }
-  
+
+  public EnumCounters<StorageType> getStorageTypeDeltas(byte storagePolicyID,
+      long dsDelta, short oldRep, short newRep) {
+    EnumCounters<StorageType> typeSpaceDeltas =
+        new EnumCounters<StorageType>(StorageType.class);
+    // Storage type and its quota are only available when storage policy is set
+    if (storagePolicyID != BlockStoragePolicySuite.ID_UNSPECIFIED) {
+      BlockStoragePolicy storagePolicy = getBlockManager().getStoragePolicy(storagePolicyID);
+
+      if (oldRep != newRep) {
+        List<StorageType> oldChosenStorageTypes =
+            storagePolicy.chooseStorageTypes(oldRep);
+
+        for (StorageType t : oldChosenStorageTypes) {
+          if (!t.supportTypeQuota()) {
+            continue;
+          }
+          Preconditions.checkArgument(dsDelta > 0);
+          typeSpaceDeltas.add(t, -dsDelta);
+        }
+      }
+
+      List<StorageType> newChosenStorageTypes =
+          storagePolicy.chooseStorageTypes(newRep);
+
+      for (StorageType t : newChosenStorageTypes) {
+        if (!t.supportTypeQuota()) {
+          continue;
+        }
+        typeSpaceDeltas.add(t, dsDelta);
+      }
+    }
+    return typeSpaceDeltas;
+  }
+
   /** Return the name of the path represented by inodes at [0, pos] */
   static String getFullPathName(INode[] inodes, int pos) {
     StringBuilder fullPathName = new StringBuilder();
@@ -751,16 +831,16 @@ public class FSDirectory implements Closeable {
    *  
    * @param iip INodes corresponding to a path
    * @param pos position where a new INode will be added
-   * @param nsDelta needed namespace
-   * @param dsDelta needed diskspace
+   * @param deltas needed namespace, diskspace and storage types
    * @param commonAncestor Last node in inodes array that is a common ancestor
    *          for a INode that is being moved from one location to the other.
    *          Pass null if a node is not being moved.
    * @throws QuotaExceededException if quota limit is exceeded.
    */
-  static void verifyQuota(INodesInPath iip, int pos, long nsDelta,
-      long dsDelta, INode commonAncestor) throws QuotaExceededException {
-    if (nsDelta <= 0 && dsDelta <= 0) {
+  static void verifyQuota(INodesInPath iip, int pos, QuotaCounts deltas,
+                          INode commonAncestor) throws QuotaExceededException {
+    if (deltas.getNameSpace() <= 0 && deltas.getDiskSpace() <= 0
+        && deltas.getTypeSpaces().allLessOrEqual(0L)) {
       // if quota is being freed or not being consumed
       return;
     }
@@ -775,7 +855,7 @@ public class FSDirectory implements Closeable {
           = iip.getINode(i).asDirectory().getDirectoryWithQuotaFeature();
       if (q != null) { // a directory with quota
         try {
-          q.verifyQuota(nsDelta, dsDelta);
+          q.verifyQuota(deltas);
         } catch (QuotaExceededException e) {
           List<INode> inodes = iip.getReadOnlyINodes();
           final String path = getFullPathName(inodes.toArray(new INode[inodes.size()]), i);
@@ -845,7 +925,7 @@ 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
@@ -873,7 +953,7 @@ public class FSDirectory implements Closeable {
     // odd. It's because a rename operation deletes the src, tries to add
     // to the dest, if that fails, re-adds the src from whence it came.
     // The rename code disables the quota when it's restoring to the
-    // original location becase a quota violation would cause the the item
+    // original location because a quota violation would cause the the item
     // to go "poof".  The fs limits must be bypassed for the same reason.
     if (checkQuota) {
       final String parentPath = existing.getPath(pos - 1);
@@ -883,21 +963,19 @@ public class FSDirectory implements Closeable {
     // always verify inode name
     verifyINodeName(inode.getLocalNameBytes());
 
-    final Quota.Counts counts = inode.computeQuotaUsage();
-    updateCount(existing, pos,
-        counts.get(Quota.NAMESPACE), counts.get(Quota.DISKSPACE), checkQuota);
+    final QuotaCounts counts = inode.computeQuotaUsage(getBlockStoragePolicySuite());
+    updateCount(existing, pos, counts, checkQuota);
+
     boolean isRename = (inode.getParent() != null);
     boolean added;
     try {
       added = parent.addChild(inode, true, existing.getLatestSnapshotId());
     } catch (QuotaExceededException e) {
-      updateCountNoQuotaCheck(existing, pos,
-          -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
+      updateCountNoQuotaCheck(existing, pos, counts.negation());
       throw e;
     }
     if (!added) {
-      updateCountNoQuotaCheck(existing, pos,
-          -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
+      updateCountNoQuotaCheck(existing, pos, counts.negation());
       return null;
     } else {
       if (!isRename) {
@@ -1023,12 +1101,13 @@ public class FSDirectory implements Closeable {
     INodeFile file = iip.getLastINode().asFile();
     int latestSnapshot = iip.getLatestSnapshotId();
     file.recordModification(latestSnapshot, true);
-    long oldDiskspace = file.diskspaceConsumed();
+    long oldDiskspaceNoRep = file.diskspaceConsumedNoReplication();
     long remainingLength =
         file.collectBlocksBeyondMax(newLength, collectedBlocks);
     file.excludeSnapshotBlocks(latestSnapshot, collectedBlocks);
     file.setModificationTime(mtime);
-    updateCount(iip, 0, file.diskspaceConsumed() - oldDiskspace, true);
+    updateCount(iip, 0, file.diskspaceConsumedNoReplication() - oldDiskspaceNoRep,
+      file.getBlockReplication(), true);
     // return whether on a block boundary
     return (remainingLength - newLength) == 0;
   }
@@ -1102,7 +1181,7 @@ public class FSDirectory implements Closeable {
     readLock();
     try {
       return rootDir.getDirectoryWithQuotaFeature().getSpaceConsumed()
-          .get(Quota.NAMESPACE);
+          .getNameSpace();
     } finally {
       readUnlock();
     }

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -85,6 +85,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV2Op;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaByStorageTypeOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetStoragePolicyOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetXAttrOp;
@@ -98,6 +99,7 @@ import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
@@ -873,6 +875,14 @@ public class FSEditLog implements LogsPurgeable {
     logEdit(op);
   }
 
+  /** Add set quota by storage type record to edit log */
+  void logSetQuotaByStorageType(String src, long dsQuota, StorageType type) {
+    SetQuotaByStorageTypeOp op = SetQuotaByStorageTypeOp.getInstance(cache.get())
+      .setSource(src)
+      .setQuotaByStorageType(dsQuota, type);
+    logEdit(op);
+  }
+
   /**  Add set permissions record to edit log */
   void logSetPermissions(String src, FsPermission permissions) {
     SetPermissionsOp op = SetPermissionsOp.getInstance(cache.get())

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

@@ -597,14 +597,14 @@ public class FSEditLogLoader {
       SetNSQuotaOp setNSQuotaOp = (SetNSQuotaOp)op;
       FSDirAttrOp.unprotectedSetQuota(
           fsDir, renameReservedPathsOnUpgrade(setNSQuotaOp.src, logVersion),
-          setNSQuotaOp.nsQuota, HdfsConstants.QUOTA_DONT_SET);
+          setNSQuotaOp.nsQuota, HdfsConstants.QUOTA_DONT_SET, null);
       break;
     }
     case OP_CLEAR_NS_QUOTA: {
       ClearNSQuotaOp clearNSQuotaOp = (ClearNSQuotaOp)op;
       FSDirAttrOp.unprotectedSetQuota(
           fsDir, renameReservedPathsOnUpgrade(clearNSQuotaOp.src, logVersion),
-          HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_DONT_SET);
+          HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_DONT_SET, null);
       break;
     }
 
@@ -612,9 +612,18 @@ public class FSEditLogLoader {
       SetQuotaOp setQuotaOp = (SetQuotaOp) op;
       FSDirAttrOp.unprotectedSetQuota(fsDir,
           renameReservedPathsOnUpgrade(setQuotaOp.src, logVersion),
-          setQuotaOp.nsQuota, setQuotaOp.dsQuota);
+          setQuotaOp.nsQuota, setQuotaOp.dsQuota, null);
       break;
 
+    case OP_SET_QUOTA_BY_STORAGETYPE:
+        FSEditLogOp.SetQuotaByStorageTypeOp setQuotaByStorageTypeOp =
+          (FSEditLogOp.SetQuotaByStorageTypeOp) op;
+        FSDirAttrOp.unprotectedSetQuota(fsDir,
+          renameReservedPathsOnUpgrade(setQuotaByStorageTypeOp.src, logVersion),
+          HdfsConstants.QUOTA_DONT_SET, setQuotaByStorageTypeOp.dsQuota,
+          setQuotaByStorageTypeOp.type);
+        break;
+
     case OP_TIMES: {
       TimesOp timesOp = (TimesOp)op;
       FSDirAttrOp.unprotectedSetTimes(

+ 85 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -64,6 +64,7 @@ import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_TRUNCAT
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_UPDATE_BLOCKS;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_UPDATE_MASTER_KEY;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_STORAGE_POLICY;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_QUOTA_BY_STORAGETYPE;
 
 import java.io.DataInput;
 import java.io.DataInputStream;
@@ -106,6 +107,7 @@ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrEditLogProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
 import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
@@ -211,6 +213,7 @@ public abstract class FSEditLogOp {
       inst.put(OP_REMOVE_XATTR, new RemoveXAttrOp());
       inst.put(OP_SET_STORAGE_POLICY, new SetStoragePolicyOp());
       inst.put(OP_APPEND, new AppendOp());
+      inst.put(OP_SET_QUOTA_BY_STORAGETYPE, new SetQuotaByStorageTypeOp());
     }
     
     public FSEditLogOp get(FSEditLogOpCodes opcode) {
@@ -2269,6 +2272,88 @@ public abstract class FSEditLogOp {
     }
   }
 
+  /** {@literal @Idempotent} for {@link ClientProtocol#setQuota} */
+  static class SetQuotaByStorageTypeOp extends FSEditLogOp {
+    String src;
+    long dsQuota;
+    StorageType type;
+
+    private SetQuotaByStorageTypeOp() {
+      super(OP_SET_QUOTA_BY_STORAGETYPE);
+    }
+
+    static SetQuotaByStorageTypeOp getInstance(OpInstanceCache cache) {
+      return (SetQuotaByStorageTypeOp)cache.get(OP_SET_QUOTA_BY_STORAGETYPE);
+    }
+
+    @Override
+    void resetSubFields() {
+      src = null;
+      dsQuota = -1L;
+      type = StorageType.DEFAULT;
+    }
+
+    SetQuotaByStorageTypeOp setSource(String src) {
+      this.src = src;
+      return this;
+    }
+
+    SetQuotaByStorageTypeOp setQuotaByStorageType(long dsQuota, StorageType type) {
+      this.type = type;
+      this.dsQuota = dsQuota;
+      return this;
+    }
+
+    @Override
+    public
+    void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeString(src, out);
+      FSImageSerialization.writeInt(type.ordinal(), out);
+      FSImageSerialization.writeLong(dsQuota, out);
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion)
+      throws IOException {
+      this.src = FSImageSerialization.readString(in);
+      this.type = StorageType.parseStorageType(FSImageSerialization.readInt(in));
+      this.dsQuota = FSImageSerialization.readLong(in);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("SetTypeQuotaOp [src=");
+      builder.append(src);
+      builder.append(", storageType=");
+      builder.append(type);
+      builder.append(", dsQuota=");
+      builder.append(dsQuota);
+      builder.append(", opCode=");
+      builder.append(opCode);
+      builder.append(", txid=");
+      builder.append(txid);
+      builder.append("]");
+      return builder.toString();
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "SRC", src);
+      XMLUtils.addSaxString(contentHandler, "STORAGETYPE",
+        Integer.toString(type.ordinal()));
+      XMLUtils.addSaxString(contentHandler, "DSQUOTA",
+        Long.toString(dsQuota));
+    }
+
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.src = st.getValue("SRC");
+      this.type = StorageType.parseStorageType(
+          Integer.parseInt(st.getValue("STORAGETYPE")));
+      this.dsQuota = Long.parseLong(st.getValue("DSQUOTA"));
+    }
+  }
+
   /** {@literal @Idempotent} for {@link ClientProtocol#setTimes} */
   static class TimesOp extends FSEditLogOp {
     int length;

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java

@@ -75,6 +75,7 @@ public enum FSEditLogOpCodes {
   OP_SET_STORAGE_POLICY         ((byte) 45),
   OP_TRUNCATE                   ((byte) 46),
   OP_APPEND                     ((byte) 47),
+  OP_SET_QUOTA_BY_STORAGETYPE   ((byte) 48),
 
   // Note that the current range of the valid OP code is 0~127
   OP_INVALID                    ((byte) -1);

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

@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -65,12 +66,13 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.util.Canceler;
+import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.util.Time;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -841,7 +843,8 @@ public class FSImage implements Closeable {
     } finally {
       FSEditLog.closeAllStreams(editStreams);
       // update the counts
-      updateCountForQuota(target.dir.rootDir);
+      updateCountForQuota(target.getBlockManager().getStoragePolicySuite(),
+          target.dir.rootDir);
     }
     prog.endPhase(Phase.LOADING_EDITS);
     return lastAppliedTxId - prevLastAppliedTxId;
@@ -855,47 +858,64 @@ public class FSImage implements Closeable {
    * This is an update of existing state of the filesystem and does not
    * throw QuotaExceededException.
    */
-  static void updateCountForQuota(INodeDirectory root) {
-    updateCountForQuotaRecursively(root, Quota.Counts.newInstance());
-  }
-  
-  private static void updateCountForQuotaRecursively(INodeDirectory dir,
-      Quota.Counts counts) {
-    final long parentNamespace = counts.get(Quota.NAMESPACE);
-    final long parentDiskspace = counts.get(Quota.DISKSPACE);
-
-    dir.computeQuotaUsage4CurrentDirectory(counts);
+  static void updateCountForQuota(BlockStoragePolicySuite bsps,
+                                  INodeDirectory root) {
+    updateCountForQuotaRecursively(bsps, root, new QuotaCounts.Builder().build());
+ }
+
+  private static void updateCountForQuotaRecursively(BlockStoragePolicySuite bsps,
+      INodeDirectory dir, QuotaCounts counts) {
+    final long parentNamespace = counts.getNameSpace();
+    final long parentDiskspace = counts.getDiskSpace();
+    final EnumCounters<StorageType> parentTypeSpaces = counts.getTypeSpaces();
+
+    dir.computeQuotaUsage4CurrentDirectory(bsps, counts);
     
     for (INode child : dir.getChildrenList(Snapshot.CURRENT_STATE_ID)) {
       if (child.isDirectory()) {
-        updateCountForQuotaRecursively(child.asDirectory(), counts);
+        updateCountForQuotaRecursively(bsps, child.asDirectory(), counts);
       } else {
         // file or symlink: count here to reduce recursive calls.
-        child.computeQuotaUsage(counts, false);
+        child.computeQuotaUsage(bsps, counts, false);
       }
     }
       
     if (dir.isQuotaSet()) {
       // check if quota is violated. It indicates a software bug.
-      final Quota.Counts q = dir.getQuotaCounts();
+      final QuotaCounts q = dir.getQuotaCounts();
 
-      final long namespace = counts.get(Quota.NAMESPACE) - parentNamespace;
-      final long nsQuota = q.get(Quota.NAMESPACE);
+      final long namespace = counts.getNameSpace() - parentNamespace;
+      final long nsQuota = q.getNameSpace();
       if (Quota.isViolated(nsQuota, namespace)) {
         LOG.error("BUG: Namespace quota violation in image for "
             + dir.getFullPathName()
             + " quota = " + nsQuota + " < consumed = " + namespace);
       }
 
-      final long diskspace = counts.get(Quota.DISKSPACE) - parentDiskspace;
-      final long dsQuota = q.get(Quota.DISKSPACE);
+      final long diskspace = counts.getDiskSpace() - parentDiskspace;
+      final long dsQuota = q.getDiskSpace();
       if (Quota.isViolated(dsQuota, diskspace)) {
         LOG.error("BUG: Diskspace quota violation in image for "
             + dir.getFullPathName()
             + " quota = " + dsQuota + " < consumed = " + diskspace);
       }
 
-      dir.getDirectoryWithQuotaFeature().setSpaceConsumed(namespace, diskspace);
+      final EnumCounters<StorageType> typeSpaces =
+          new EnumCounters<StorageType>(StorageType.class);
+      for (StorageType t : StorageType.getTypesSupportingQuota()) {
+        final long typeSpace = counts.getTypeSpaces().get(t) -
+            parentTypeSpaces.get(t);
+        final long typeQuota = q.getTypeSpaces().get(t);
+        if (Quota.isViolated(typeQuota, typeSpace)) {
+          LOG.error("BUG Disk quota by storage type violation in image for "
+              + dir.getFullPathName()
+              + " type = " + t.toString() + " quota = "
+              + typeQuota + " < consumed " + typeSpace);
+        }
+      }
+
+      dir.getDirectoryWithQuotaFeature().setSpaceConsumed(namespace, diskspace,
+          typeSpaces);
     }
   }
 

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

@@ -448,9 +448,9 @@ public class FSImageFormat {
 
   /** Update the root node's attributes */
   private void updateRootAttr(INodeWithAdditionalFields root) {                                                           
-    final Quota.Counts q = root.getQuotaCounts();
-    final long nsQuota = q.get(Quota.NAMESPACE);
-    final long dsQuota = q.get(Quota.DISKSPACE);
+    final QuotaCounts q = root.getQuotaCounts();
+    final long nsQuota = q.getNameSpace();
+    final long dsQuota = q.getDiskSpace();
     FSDirectory fsDir = namesystem.dir;
     if (nsQuota != -1 || dsQuota != -1) {
       fsDir.rootDir.getDirectoryWithQuotaFeature().setQuota(nsQuota, dsQuota);
@@ -825,7 +825,8 @@ public class FSImageFormat {
       final INodeDirectory dir = new INodeDirectory(inodeId, localName,
           permissions, modificationTime);
       if (nsQuota >= 0 || dsQuota >= 0) {
-        dir.addDirectoryWithQuotaFeature(nsQuota, dsQuota);
+        dir.addDirectoryWithQuotaFeature(new DirectoryWithQuotaFeature.Builder().
+            nameSpaceQuota(nsQuota).spaceQuota(dsQuota).build());
       }
       if (withSnapshot) {
         dir.addSnapshotFeature(null);
@@ -912,11 +913,11 @@ public class FSImageFormat {
       //read quotas
       final long nsQuota = in.readLong();
       final long dsQuota = in.readLong();
-  
+
       return nsQuota == -1L && dsQuota == -1L ? new INodeDirectoryAttributes.SnapshotCopy(
           name, permissions, null, modificationTime, null)
         : new INodeDirectoryAttributes.CopyWithQuota(name, permissions,
-            null, modificationTime, nsQuota, dsQuota, null);
+            null, modificationTime, nsQuota, dsQuota, null, null);
     }
   
     private void loadFilesUnderConstruction(DataInput in,
@@ -1234,7 +1235,7 @@ public class FSImageFormat {
       final FSNamesystem sourceNamesystem = context.getSourceNamesystem();
       final INodeDirectory rootDir = sourceNamesystem.dir.rootDir;
       final long numINodes = rootDir.getDirectoryWithQuotaFeature()
-          .getSpaceConsumed().get(Quota.NAMESPACE);
+          .getSpaceConsumed().getNameSpace();
       String sdPath = newFile.getParentFile().getParentFile().getAbsolutePath();
       Step step = new Step(StepType.INODES, sdPath);
       StartupProgress prog = NameNode.getStartupProgress();

+ 72 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java

@@ -37,7 +37,9 @@ 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.fs.XAttr;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
@@ -52,7 +54,10 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrCompactProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrFeatureProto;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.QuotaByStorageTypeEntryProto;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.QuotaByStorageTypeFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.base.Preconditions;
@@ -140,6 +145,18 @@ public final class FSImageFormatPBINode {
       return b.build();
     }
 
+    public static ImmutableList<QuotaByStorageTypeEntry> loadQuotaByStorageTypeEntries(
+      QuotaByStorageTypeFeatureProto proto) {
+      ImmutableList.Builder<QuotaByStorageTypeEntry> b = ImmutableList.builder();
+      for (QuotaByStorageTypeEntryProto quotaEntry : proto.getQuotasList()) {
+        StorageType type = PBHelper.convertStorageType(quotaEntry.getStorageType());
+        long quota = quotaEntry.getQuota();
+        b.add(new QuotaByStorageTypeEntry.Builder().setStorageType(type)
+            .setQuota(quota).build());
+      }
+      return b.build();
+    }
+
     public static INodeDirectory loadINodeDirectory(INodeSection.INode n,
         LoaderContext state) {
       assert n.getType() == INodeSection.INode.Type.DIRECTORY;
@@ -149,10 +166,33 @@ public final class FSImageFormatPBINode {
           state.getStringTable());
       final INodeDirectory dir = new INodeDirectory(n.getId(), n.getName()
           .toByteArray(), permissions, d.getModificationTime());
-
       final long nsQuota = d.getNsQuota(), dsQuota = d.getDsQuota();
       if (nsQuota >= 0 || dsQuota >= 0) {
-        dir.addDirectoryWithQuotaFeature(nsQuota, dsQuota);
+        dir.addDirectoryWithQuotaFeature(new DirectoryWithQuotaFeature.Builder().
+            nameSpaceQuota(nsQuota).spaceQuota(dsQuota).build());
+      }
+      EnumCounters<StorageType> typeQuotas = null;
+      if (d.hasTypeQuotas()) {
+        ImmutableList<QuotaByStorageTypeEntry> qes =
+            loadQuotaByStorageTypeEntries(d.getTypeQuotas());
+        typeQuotas = new EnumCounters<StorageType>(StorageType.class,
+            HdfsConstants.QUOTA_RESET);
+        for (QuotaByStorageTypeEntry qe : qes) {
+          if (qe.getQuota() >= 0 && qe.getStorageType() != null &&
+              qe.getStorageType().supportTypeQuota()) {
+            typeQuotas.set(qe.getStorageType(), qe.getQuota());
+          }
+        }
+
+        if (typeQuotas.anyGreaterOrEqual(0)) {
+          DirectoryWithQuotaFeature q = dir.getDirectoryWithQuotaFeature();
+          if (q == null) {
+            dir.addDirectoryWithQuotaFeature(new DirectoryWithQuotaFeature.
+                Builder().typeQuotas(typeQuotas).build());
+          } else {
+            q.setQuota(typeQuotas);
+          }
+        }
       }
 
       if (d.hasAcl()) {
@@ -332,12 +372,16 @@ public final class FSImageFormatPBINode {
 
     private void loadRootINode(INodeSection.INode p) {
       INodeDirectory root = loadINodeDirectory(p, parent.getLoaderContext());
-      final Quota.Counts q = root.getQuotaCounts();
-      final long nsQuota = q.get(Quota.NAMESPACE);
-      final long dsQuota = q.get(Quota.DISKSPACE);
+      final QuotaCounts q = root.getQuotaCounts();
+      final long nsQuota = q.getNameSpace();
+      final long dsQuota = q.getDiskSpace();
       if (nsQuota != -1 || dsQuota != -1) {
         dir.rootDir.getDirectoryWithQuotaFeature().setQuota(nsQuota, dsQuota);
       }
+      final EnumCounters<StorageType> typeQuotas = q.getTypeSpaces();
+      if (typeQuotas.anyGreaterOrEqual(0)) {
+        dir.rootDir.getDirectoryWithQuotaFeature().setQuota(typeQuotas);
+      }
       dir.rootDir.cloneModificationTime(root);
       dir.rootDir.clonePermissionStatus(root);
       // root dir supports having extended attributes according to POSIX
@@ -396,6 +440,22 @@ public final class FSImageFormatPBINode {
       return b;
     }
 
+    private static QuotaByStorageTypeFeatureProto.Builder
+        buildQuotaByStorageTypeEntries(QuotaCounts q) {
+      QuotaByStorageTypeFeatureProto.Builder b =
+          QuotaByStorageTypeFeatureProto.newBuilder();
+      for (StorageType t: StorageType.getTypesSupportingQuota()) {
+        if (q.getTypeSpace(t) >= 0) {
+          QuotaByStorageTypeEntryProto.Builder eb =
+              QuotaByStorageTypeEntryProto.newBuilder().
+              setStorageType(PBHelper.convertStorageType(t)).
+              setQuota(q.getTypeSpace(t));
+          b.addQuotas(eb);
+        }
+      }
+      return b;
+    }
+
     public static INodeSection.INodeFile.Builder buildINodeFile(
         INodeFileAttributes file, final SaverContext state) {
       INodeSection.INodeFile.Builder b = INodeSection.INodeFile.newBuilder()
@@ -419,13 +479,17 @@ public final class FSImageFormatPBINode {
 
     public static INodeSection.INodeDirectory.Builder buildINodeDirectory(
         INodeDirectoryAttributes dir, final SaverContext state) {
-      Quota.Counts quota = dir.getQuotaCounts();
+      QuotaCounts quota = dir.getQuotaCounts();
       INodeSection.INodeDirectory.Builder b = INodeSection.INodeDirectory
           .newBuilder().setModificationTime(dir.getModificationTime())
-          .setNsQuota(quota.get(Quota.NAMESPACE))
-          .setDsQuota(quota.get(Quota.DISKSPACE))
+          .setNsQuota(quota.getNameSpace())
+          .setDsQuota(quota.getDiskSpace())
           .setPermission(buildPermissionStatus(dir, state.getStringMap()));
 
+      if (quota.getTypeSpaces().anyGreaterOrEqual(0)) {
+        b.setTypeQuotas(buildQuotaByStorageTypeEntries(quota));
+      }
+
       AclFeature f = dir.getAclFeature();
       if (f != null) {
         b.setAcl(buildAclEntries(f, state.getStringMap()));

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

@@ -221,10 +221,10 @@ public class FSImageSerialization {
     out.writeLong(file.getPreferredBlockSize());
   }
 
-  private static void writeQuota(Quota.Counts quota, DataOutput out)
+  private static void writeQuota(QuotaCounts quota, DataOutput out)
       throws IOException {
-    out.writeLong(quota.get(Quota.NAMESPACE));
-    out.writeLong(quota.get(Quota.DISKSPACE));
+    out.writeLong(quota.getNameSpace());
+    out.writeLong(quota.getDiskSpace());
   }
 
   /**

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

@@ -254,6 +254,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
@@ -2117,7 +2118,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // update the quota: use the preferred block size for UC block
     final long diff =
         file.getPreferredBlockSize() - truncatedBlockUC.getNumBytes();
-    dir.updateSpaceConsumed(iip, 0, diff * file.getBlockReplication());
+    dir.updateSpaceConsumed(iip, 0, diff, file.getBlockReplication());
     return newBlock;
   }
 
@@ -2664,7 +2665,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (ret != null) {
         // update the quota: use the preferred block size for UC block
         final long diff = file.getPreferredBlockSize() - ret.getBlockSize();
-        dir.updateSpaceConsumed(iip, 0, diff * file.getBlockReplication());
+        dir.updateSpaceConsumed(iip, 0, diff, file.getBlockReplication());
       }
     } else {
       BlockInfoContiguous lastBlock = file.getLastBlock();
@@ -3802,19 +3803,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /**
    * Set the namespace quota and diskspace quota for a directory.
-   * See {@link ClientProtocol#setQuota(String, long, long)} for the 
+   * See {@link ClientProtocol#setQuota(String, long, long, StorageType)} for the
    * contract.
    * 
    * Note: This does not support ".inodes" relative path.
    */
-  void setQuota(String src, long nsQuota, long dsQuota)
+  void setQuota(String src, long nsQuota, long dsQuota, StorageType type)
       throws IOException {
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set quota on " + src);
-      FSDirAttrOp.setQuota(dir, src, nsQuota, dsQuota);
+      FSDirAttrOp.setQuota(dir, src, nsQuota, dsQuota, type);
     } finally {
       writeUnlock();
     }
@@ -4041,7 +4042,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();    
     if (diff > 0) {
       try {
-        dir.updateSpaceConsumed(iip, 0, -diff*fileINode.getFileReplication());
+        dir.updateSpaceConsumed(iip, 0, -diff, fileINode.getFileReplication());
       } catch (IOException e) {
         LOG.warn("Unexpected exception while updating disk space.", e);
       }

+ 46 - 30
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -384,7 +385,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * 2.4 To clean {@link INodeDirectory} with snapshot: delete the corresponding 
    * snapshot in its diff list. Recursively clean its children.
    * </pre>
-   * 
+   *
+   * @param bsps
+   *          block storage policy suite to calculate intended storage type usage
    * @param snapshotId
    *          The id of the snapshot to delete. 
    *          {@link Snapshot#CURRENT_STATE_ID} means to delete the current
@@ -401,7 +404,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    *          inodeMap
    * @return quota usage delta when deleting a snapshot
    */
-  public abstract Quota.Counts cleanSubtree(final int snapshotId,
+  public abstract QuotaCounts cleanSubtree(final BlockStoragePolicySuite bsps,
+      final int snapshotId,
       int priorSnapshotId, BlocksMapUpdateInfo collectedBlocks,
       List<INode> removedINodes);
   
@@ -411,7 +415,11 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * directory, the method goes down the subtree and collects blocks from the
    * descents, and clears its parent/children references as well. The method
    * also clears the diff list if the INode contains snapshot diff list.
-   * 
+   *
+   * @param bsps
+   *          block storage policy suite to calculate intended storage type usage
+   *          This is needed because INodeReference#destroyAndCollectBlocks() needs
+   *          to call INode#cleanSubtree(), which calls INode#computeQuotaUsage().
    * @param collectedBlocks
    *          blocks collected from the descents for further block
    *          deletion/update will be added to this map.
@@ -420,6 +428,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    *          inodeMap
    */
   public abstract void destroyAndCollectBlocks(
+      BlockStoragePolicySuite bsps,
       BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes);
 
   /** Compute {@link ContentSummary}. Blocking call */
@@ -434,11 +443,12 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   public final ContentSummary computeAndConvertContentSummary(
       ContentSummaryComputationContext summary) {
     Content.Counts counts = computeContentSummary(summary).getCounts();
-    final Quota.Counts q = getQuotaCounts();
+    final QuotaCounts q = getQuotaCounts();
     return new ContentSummary(counts.get(Content.LENGTH),
         counts.get(Content.FILE) + counts.get(Content.SYMLINK),
-        counts.get(Content.DIRECTORY), q.get(Quota.NAMESPACE),
-        counts.get(Content.DISKSPACE), q.get(Quota.DISKSPACE));
+        counts.get(Content.DIRECTORY), q.getNameSpace(),
+        counts.get(Content.DISKSPACE), q.getDiskSpace());
+    // TODO: storage type quota reporting HDFS-7701.
   }
 
   /**
@@ -450,24 +460,24 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   public abstract ContentSummaryComputationContext computeContentSummary(
       ContentSummaryComputationContext summary);
 
-  
+
   /**
-   * Check and add namespace/diskspace consumed to itself and the ancestors.
+   * Check and add namespace/diskspace/storagetype consumed to itself and the ancestors.
    * @throws QuotaExceededException if quote is violated.
    */
-  public void addSpaceConsumed(long nsDelta, long dsDelta, boolean verify) 
-      throws QuotaExceededException {
-    addSpaceConsumed2Parent(nsDelta, dsDelta, verify);
+  public void addSpaceConsumed(QuotaCounts counts, boolean verify)
+    throws QuotaExceededException {
+    addSpaceConsumed2Parent(counts, verify);
   }
 
   /**
-   * Check and add namespace/diskspace consumed to itself and the ancestors.
+   * Check and add namespace/diskspace/storagetype consumed to itself and the ancestors.
    * @throws QuotaExceededException if quote is violated.
    */
-  void addSpaceConsumed2Parent(long nsDelta, long dsDelta, boolean verify) 
-      throws QuotaExceededException {
+  void addSpaceConsumed2Parent(QuotaCounts counts, boolean verify)
+    throws QuotaExceededException {
     if (parent != null) {
-      parent.addSpaceConsumed(nsDelta, dsDelta, verify);
+      parent.addSpaceConsumed(counts, verify);
     }
   }
 
@@ -475,20 +485,24 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * Get the quota set for this inode
    * @return the quota counts.  The count is -1 if it is not set.
    */
-  public Quota.Counts getQuotaCounts() {
-    return Quota.Counts.newInstance(-1, -1);
+  public QuotaCounts getQuotaCounts() {
+    return new QuotaCounts.Builder().
+        nameCount(HdfsConstants.QUOTA_RESET).
+        spaceCount(HdfsConstants.QUOTA_RESET).
+        typeCounts(HdfsConstants.QUOTA_RESET).
+        build();
   }
-  
+
   public final boolean isQuotaSet() {
-    final Quota.Counts q = getQuotaCounts();
-    return q.get(Quota.NAMESPACE) >= 0 || q.get(Quota.DISKSPACE) >= 0;
+    final QuotaCounts qc = getQuotaCounts();
+    return qc.anyNsSpCountGreaterOrEqual(0) || qc.anyTypeCountGreaterOrEqual(0);
   }
-  
+
   /**
    * Count subtree {@link Quota#NAMESPACE} and {@link Quota#DISKSPACE} usages.
    */
-  public final Quota.Counts computeQuotaUsage() {
-    return computeQuotaUsage(new Quota.Counts(), true);
+  public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
+    return computeQuotaUsage(bsps, new QuotaCounts.Builder().build(), true);
   }
 
   /**
@@ -511,7 +525,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * creation time of the snapshot associated with the {@link WithName} node.
    * We do not count in the size of the diff list.  
    * <pre>
-   * 
+   *
+   * @param bsps Block storage policy suite to calculate intended storage type usage
    * @param counts The subtree counts for returning.
    * @param useCache Whether to use cached quota usage. Note that 
    *                 {@link WithName} node never uses cache for its subtree.
@@ -521,14 +536,15 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    *                       {@link WithName} node.
    * @return The same objects as the counts parameter.
    */
-  public abstract Quota.Counts computeQuotaUsage(Quota.Counts counts,
-      boolean useCache, int lastSnapshotId);
+  public abstract QuotaCounts computeQuotaUsage(
+    BlockStoragePolicySuite bsps,
+    QuotaCounts counts, boolean useCache, int lastSnapshotId);
 
-  public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
-      boolean useCache) {
-    return computeQuotaUsage(counts, useCache, Snapshot.CURRENT_STATE_ID);
+  public final QuotaCounts computeQuotaUsage(
+    BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache) {
+    return computeQuotaUsage(bsps, counts, useCache, Snapshot.CURRENT_STATE_ID);
   }
-  
+
   /**
    * @return null if the local name is null; otherwise, return the local name.
    */

+ 61 - 47
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFea
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
@@ -136,35 +137,45 @@ public class INodeDirectory extends INodeWithAdditionalFields
         BlockStoragePolicySuite.ID_UNSPECIFIED;
   }
 
-  void setQuota(long nsQuota, long dsQuota) {
+  void setQuota(BlockStoragePolicySuite bsps, long nsQuota, long dsQuota, StorageType type) {
     DirectoryWithQuotaFeature quota = getDirectoryWithQuotaFeature();
     if (quota != null) {
       // already has quota; so set the quota to the new values
-      quota.setQuota(nsQuota, dsQuota);
+      if (type != null) {
+        quota.setQuota(dsQuota, type);
+      } else {
+        quota.setQuota(nsQuota, dsQuota);
+      }
       if (!isQuotaSet() && !isRoot()) {
         removeFeature(quota);
       }
     } else {
-      final Quota.Counts c = computeQuotaUsage();
-      quota = addDirectoryWithQuotaFeature(nsQuota, dsQuota);
-      quota.setSpaceConsumed(c.get(Quota.NAMESPACE), c.get(Quota.DISKSPACE));
+      final QuotaCounts c = computeQuotaUsage(bsps);
+      DirectoryWithQuotaFeature.Builder builder =
+          new DirectoryWithQuotaFeature.Builder().nameSpaceQuota(nsQuota);
+      if (type != null) {
+        builder.typeQuota(type, dsQuota);
+      } else {
+        builder.spaceQuota(dsQuota);
+      }
+      addDirectoryWithQuotaFeature(builder.build()).setSpaceConsumed(c);
     }
   }
 
   @Override
-  public Quota.Counts getQuotaCounts() {
+  public QuotaCounts getQuotaCounts() {
     final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
     return q != null? q.getQuota(): super.getQuotaCounts();
   }
 
   @Override
-  public void addSpaceConsumed(long nsDelta, long dsDelta, boolean verify) 
-      throws QuotaExceededException {
+  public void addSpaceConsumed(QuotaCounts counts, boolean verify)
+    throws QuotaExceededException {
     final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
     if (q != null) {
-      q.addSpaceConsumed(this, nsDelta, dsDelta, verify);
+      q.addSpaceConsumed(this, counts, verify);
     } else {
-      addSpaceConsumed2Parent(nsDelta, dsDelta, verify);
+      addSpaceConsumed2Parent(counts, verify);
     }
   }
 
@@ -182,12 +193,10 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   DirectoryWithQuotaFeature addDirectoryWithQuotaFeature(
-      long nsQuota, long dsQuota) {
+      DirectoryWithQuotaFeature q) {
     Preconditions.checkState(!isWithQuota(), "Directory is already with quota");
-    final DirectoryWithQuotaFeature quota = new DirectoryWithQuotaFeature(
-        nsQuota, dsQuota);
-    addFeature(quota);
-    return quota;
+    addFeature(q);
+    return q;
   }
 
   int searchChildren(byte[] name) {
@@ -254,10 +263,10 @@ public class INodeDirectory extends INodeWithAdditionalFields
     return getDirectorySnapshottableFeature().addSnapshot(this, id, name);
   }
 
-  public Snapshot removeSnapshot(String snapshotName,
+  public Snapshot removeSnapshot(BlockStoragePolicySuite bsps, String snapshotName,
       BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
       throws SnapshotException {
-    return getDirectorySnapshottableFeature().removeSnapshot(this,
+    return getDirectorySnapshottableFeature().removeSnapshot(bsps, this,
         snapshotName, collectedBlocks, removedINodes);
   }
 
@@ -559,7 +568,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   @Override
-  public Quota.Counts computeQuotaUsage(Quota.Counts counts, boolean useCache,
+  public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache,
       int lastSnapshotId) {
     final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     
@@ -570,9 +579,9 @@ public class INodeDirectory extends INodeWithAdditionalFields
         && !(useCache && isQuotaSet())) {
       ReadOnlyList<INode> childrenList = getChildrenList(lastSnapshotId);
       for (INode child : childrenList) {
-        child.computeQuotaUsage(counts, useCache, lastSnapshotId);
+        child.computeQuotaUsage(bsps, counts, useCache, lastSnapshotId);
       }
-      counts.add(Quota.NAMESPACE, 1);
+      counts.addNameSpace(1);
       return counts;
     }
     
@@ -582,27 +591,28 @@ public class INodeDirectory extends INodeWithAdditionalFields
       return q.addNamespaceDiskspace(counts);
     } else {
       useCache = q != null && !q.isQuotaSet() ? false : useCache;
-      return computeDirectoryQuotaUsage(counts, useCache, lastSnapshotId);
+      return computeDirectoryQuotaUsage(bsps, counts, useCache, lastSnapshotId);
     }
   }
 
-  private Quota.Counts computeDirectoryQuotaUsage(Quota.Counts counts,
-      boolean useCache, int lastSnapshotId) {
+  private QuotaCounts computeDirectoryQuotaUsage(BlockStoragePolicySuite bsps,
+      QuotaCounts counts, boolean useCache, int lastSnapshotId) {
     if (children != null) {
       for (INode child : children) {
-        child.computeQuotaUsage(counts, useCache, lastSnapshotId);
+        child.computeQuotaUsage(bsps, counts, useCache, lastSnapshotId);
       }
     }
-    return computeQuotaUsage4CurrentDirectory(counts);
+    return computeQuotaUsage4CurrentDirectory(bsps, counts);
   }
   
   /** Add quota usage for this inode excluding children. */
-  public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
-    counts.add(Quota.NAMESPACE, 1);
+  public QuotaCounts computeQuotaUsage4CurrentDirectory(
+      BlockStoragePolicySuite bsps, QuotaCounts counts) {
+    counts.addNameSpace(1);
     // include the diff list
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     if (sf != null) {
-      sf.computeQuotaUsage4CurrentDirectory(counts);
+      sf.computeQuotaUsage4CurrentDirectory(bsps, counts);
     }
     return counts;
   }
@@ -612,7 +622,8 @@ public class INodeDirectory extends INodeWithAdditionalFields
       ContentSummaryComputationContext summary) {
     final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     if (sf != null) {
-      sf.computeContentSummary4Snapshot(summary.getCounts());
+      sf.computeContentSummary4Snapshot(summary.getBlockStoragePolicySuite(),
+          summary.getCounts());
     }
     final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
     if (q != null) {
@@ -702,7 +713,8 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * (with OVERWRITE option) removes a file/dir from the dst tree, add it back
    * and delete possible record in the deleted list.  
    */
-  public void undoRename4DstParent(final INode deletedChild,
+  public void undoRename4DstParent(final BlockStoragePolicySuite bsps,
+      final INode deletedChild,
       int latestSnapshotId) throws QuotaExceededException {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     Preconditions.checkState(sf != null,
@@ -714,9 +726,9 @@ public class INodeDirectory extends INodeWithAdditionalFields
     // update quota usage if adding is successfully and the old child has not
     // been stored in deleted list before
     if (added && !removeDeletedChild) {
-      final Quota.Counts counts = deletedChild.computeQuotaUsage();
-      addSpaceConsumed(counts.get(Quota.NAMESPACE),
-          counts.get(Quota.DISKSPACE), false);
+      final QuotaCounts counts = deletedChild.computeQuotaUsage(bsps);
+      addSpaceConsumed(counts, false);
+
     }
   }
 
@@ -732,10 +744,11 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   /** Call cleanSubtree(..) recursively down the subtree. */
-  public Quota.Counts cleanSubtreeRecursively(final int snapshot,
+  public QuotaCounts cleanSubtreeRecursively(final BlockStoragePolicySuite bsps,
+      final int snapshot,
       int prior, final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes, final Map<INode, INode> excludedNodes) {
-    Quota.Counts counts = Quota.Counts.newInstance();
+    QuotaCounts counts = new QuotaCounts.Builder().build();
     // in case of deletion snapshot, since this call happens after we modify
     // the diff list, the snapshot to be deleted has been combined or renamed
     // to its latest previous snapshot. (besides, we also need to consider nodes
@@ -748,7 +761,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
           && excludedNodes.containsKey(child)) {
         continue;
       } else {
-        Quota.Counts childCounts = child.cleanSubtree(snapshot, prior,
+        QuotaCounts childCounts = child.cleanSubtree(bsps, snapshot, prior,
             collectedBlocks, removedINodes);
         counts.add(childCounts);
       }
@@ -757,14 +770,15 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   @Override
-  public void destroyAndCollectBlocks(final BlocksMapUpdateInfo collectedBlocks,
+  public void destroyAndCollectBlocks(final BlockStoragePolicySuite bsps,
+      final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
     final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     if (sf != null) {
-      sf.clear(this, collectedBlocks, removedINodes);
+      sf.clear(bsps, this, collectedBlocks, removedINodes);
     }
     for (INode child : getChildrenList(Snapshot.CURRENT_STATE_ID)) {
-      child.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+      child.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes);
     }
     if (getAclFeature() != null) {
       AclStorage.removeAclFeature(getAclFeature());
@@ -774,30 +788,30 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
   
   @Override
-  public Quota.Counts cleanSubtree(final int snapshotId, int priorSnapshotId,
+  public QuotaCounts cleanSubtree(final BlockStoragePolicySuite bsps,
+      final int snapshotId, int priorSnapshotId,
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     // there is snapshot data
     if (sf != null) {
-      return sf.cleanDirectory(this, snapshotId, priorSnapshotId,
+      return sf.cleanDirectory(bsps, this, snapshotId, priorSnapshotId,
           collectedBlocks, removedINodes);
     }
     // there is no snapshot data
     if (priorSnapshotId == Snapshot.NO_SNAPSHOT_ID
         && snapshotId == Snapshot.CURRENT_STATE_ID) {
       // destroy the whole subtree and collect blocks that should be deleted
-      Quota.Counts counts = Quota.Counts.newInstance();
-      this.computeQuotaUsage(counts, true);
-      destroyAndCollectBlocks(collectedBlocks, removedINodes);
+      QuotaCounts counts = new QuotaCounts.Builder().build();
+      this.computeQuotaUsage(bsps, counts, true);
+      destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes);
       return counts; 
     } else {
       // process recursively down the subtree
-      Quota.Counts counts = cleanSubtreeRecursively(snapshotId, priorSnapshotId,
+      QuotaCounts counts = cleanSubtreeRecursively(bsps, snapshotId, priorSnapshotId,
           collectedBlocks, removedINodes, null);
       if (isQuotaSet()) {
-        getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
-            -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
+        getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(counts.negation());
       }
       return counts;
     }

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

@@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.util.EnumCounters;
 
 import com.google.common.base.Preconditions;
 
@@ -27,7 +29,7 @@ import com.google.common.base.Preconditions;
  */
 @InterfaceAudience.Private
 public interface INodeDirectoryAttributes extends INodeAttributes {
-  public Quota.Counts getQuotaCounts();
+  public QuotaCounts getQuotaCounts();
 
   public boolean metadataEquals(INodeDirectoryAttributes other);
   
@@ -45,8 +47,9 @@ public interface INodeDirectoryAttributes extends INodeAttributes {
     }
 
     @Override
-    public Quota.Counts getQuotaCounts() {
-      return Quota.Counts.newInstance(-1, -1);
+    public QuotaCounts getQuotaCounts() {
+      return new QuotaCounts.Builder().nameCount(-1).
+          spaceCount(-1).typeCounts(-1).build();
     }
 
     @Override
@@ -60,29 +63,26 @@ public interface INodeDirectoryAttributes extends INodeAttributes {
   }
 
   public static class CopyWithQuota extends INodeDirectoryAttributes.SnapshotCopy {
-    private final long nsQuota;
-    private final long dsQuota;
-
+    private QuotaCounts quota;
 
     public CopyWithQuota(byte[] name, PermissionStatus permissions,
         AclFeature aclFeature, long modificationTime, long nsQuota,
-        long dsQuota, XAttrFeature xAttrsFeature) {
+        long dsQuota, EnumCounters<StorageType> typeQuotas, XAttrFeature xAttrsFeature) {
       super(name, permissions, aclFeature, modificationTime, xAttrsFeature);
-      this.nsQuota = nsQuota;
-      this.dsQuota = dsQuota;
+      this.quota = new QuotaCounts.Builder().nameCount(nsQuota).
+          spaceCount(dsQuota).typeCounts(typeQuotas).build();
     }
 
     public CopyWithQuota(INodeDirectory dir) {
       super(dir);
       Preconditions.checkArgument(dir.isQuotaSet());
-      final Quota.Counts q = dir.getQuotaCounts();
-      this.nsQuota = q.get(Quota.NAMESPACE);
-      this.dsQuota = q.get(Quota.DISKSPACE);
+      final QuotaCounts q = dir.getQuotaCounts();
+      this.quota = new QuotaCounts.Builder().quotaCount(q).build();
     }
-    
+
     @Override
-    public Quota.Counts getQuotaCounts() {
-      return Quota.Counts.newInstance(nsQuota, dsQuota);
+    public QuotaCounts getQuotaCounts() {
+      return new QuotaCounts.Builder().quotaCount(quota).build();
     }
   }
 }

+ 63 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -31,6 +31,7 @@ import java.util.Set;
 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.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.util.LongBitFormat;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -488,21 +490,22 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override
-  public Quota.Counts cleanSubtree(final int snapshot, int priorSnapshotId,
+  public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, final int snapshot,
+                                  int priorSnapshotId,
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf != null) {
-      return sf.cleanFile(this, snapshot, priorSnapshotId, collectedBlocks,
+      return sf.cleanFile(bsps, this, snapshot, priorSnapshotId, collectedBlocks,
           removedINodes);
     }
-    Quota.Counts counts = Quota.Counts.newInstance();
+    QuotaCounts counts = new QuotaCounts.Builder().build();
     if (snapshot == CURRENT_STATE_ID) {
       if (priorSnapshotId == NO_SNAPSHOT_ID) {
         // this only happens when deleting the current file and the file is not
         // in any snapshot
-        computeQuotaUsage(counts, false);
-        destroyAndCollectBlocks(collectedBlocks, removedINodes);
+        computeQuotaUsage(bsps, counts, false);
+        destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes);
       } else {
         // when deleting the current file and the file is in snapshot, we should
         // clean the 0-sized block if the file is UC
@@ -516,8 +519,8 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override
-  public void destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes) {
+  public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
+      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
     if (blocks != null && collectedBlocks != null) {
       for (BlockInfoContiguous blk : blocks) {
         collectedBlocks.addDeleteBlock(blk);
@@ -543,11 +546,15 @@ public class INodeFile extends INodeWithAdditionalFields
     return getFullPathName();
   }
 
+  // This is the only place that needs to use the BlockStoragePolicySuite to
+  // derive the intended storage type usage for quota by storage type
   @Override
-  public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
-      boolean useCache, int lastSnapshotId) {
+  public final QuotaCounts computeQuotaUsage(
+      BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache,
+      int lastSnapshotId) {
     long nsDelta = 1;
-    final long dsDelta;
+    final long dsDeltaNoReplication;
+    short dsReplication;
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf != null) {
       FileDiffList fileDiffList = sf.getDiffs();
@@ -555,18 +562,33 @@ public class INodeFile extends INodeWithAdditionalFields
 
       if (lastSnapshotId == Snapshot.CURRENT_STATE_ID
           || last == Snapshot.CURRENT_STATE_ID) {
-        dsDelta = diskspaceConsumed();
+        dsDeltaNoReplication = diskspaceConsumedNoReplication();
+        dsReplication = getBlockReplication();
       } else if (last < lastSnapshotId) {
-        dsDelta = computeFileSize(true, false) * getFileReplication();
-      } else {      
+        dsDeltaNoReplication = computeFileSize(true, false);
+        dsReplication = getFileReplication();
+      } else {
         int sid = fileDiffList.getSnapshotById(lastSnapshotId);
-        dsDelta = diskspaceConsumed(sid);
+        dsDeltaNoReplication = diskspaceConsumedNoReplication(sid);
+        dsReplication = getReplication(sid);
       }
     } else {
-      dsDelta = diskspaceConsumed();
+      dsDeltaNoReplication = diskspaceConsumedNoReplication();
+      dsReplication = getBlockReplication();
+    }
+    counts.addNameSpace(nsDelta);
+    counts.addDiskSpace(dsDeltaNoReplication * dsReplication);
+
+    if (getStoragePolicyID() != BlockStoragePolicySuite.ID_UNSPECIFIED){
+      BlockStoragePolicy bsp = bsps.getPolicy(getStoragePolicyID());
+      List<StorageType> storageTypes = bsp.chooseStorageTypes(dsReplication);
+      for (StorageType t : storageTypes) {
+        if (!t.supportTypeQuota()) {
+          continue;
+        }
+        counts.addTypeSpace(t, dsDeltaNoReplication);
+      }
     }
-    counts.add(Quota.NAMESPACE, nsDelta);
-    counts.add(Quota.DISKSPACE, dsDelta);
     return counts;
   }
 
@@ -660,9 +682,13 @@ public class INodeFile extends INodeWithAdditionalFields
    * Use preferred block size for the last block if it is under construction.
    */
   public final long diskspaceConsumed() {
+    return diskspaceConsumedNoReplication() * getBlockReplication();
+  }
+
+  public final long diskspaceConsumedNoReplication() {
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if(sf == null) {
-      return computeFileSize(true, true) * getBlockReplication();
+      return computeFileSize(true, true);
     }
 
     // Collect all distinct blocks
@@ -684,18 +710,34 @@ public class INodeFile extends INodeWithAdditionalFields
         lastBlock instanceof BlockInfoContiguousUnderConstruction) {
       size += getPreferredBlockSize() - lastBlock.getNumBytes();
     }
-    return size * getBlockReplication();
+    return size;
   }
 
   public final long diskspaceConsumed(int lastSnapshotId) {
     if (lastSnapshotId != CURRENT_STATE_ID) {
       return computeFileSize(lastSnapshotId)
-          * getFileReplication(lastSnapshotId);
+        * getFileReplication(lastSnapshotId);
     } else {
       return diskspaceConsumed();
     }
   }
-  
+
+  public final short getReplication(int lastSnapshotId) {
+    if (lastSnapshotId != CURRENT_STATE_ID) {
+      return getFileReplication(lastSnapshotId);
+    } else {
+      return getBlockReplication();
+    }
+  }
+
+  public final long diskspaceConsumedNoReplication(int lastSnapshotId) {
+    if (lastSnapshotId != CURRENT_STATE_ID) {
+      return computeFileSize(lastSnapshotId);
+    } else {
+      return diskspaceConsumedNoReplication();
+    }
+  }
+
   /**
    * Return the penultimate allocated block for this file.
    */

+ 10 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java

@@ -23,7 +23,6 @@ import java.util.List;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
-import org.apache.hadoop.hdfs.server.namenode.Quota.Counts;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.LightWeightGSet;
 
@@ -97,17 +96,18 @@ public class INodeMap {
       }
       
       @Override
-      public void destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks,
-          List<INode> removedINodes) {
+      public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
+          BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes) {
         // Nothing to do
       }
-      
+
       @Override
-      public Counts computeQuotaUsage(Counts counts, boolean useCache,
-          int lastSnapshotId) {
+      public QuotaCounts computeQuotaUsage(
+          BlockStoragePolicySuite bsps, QuotaCounts counts,
+          boolean useCache, int lastSnapshotId) {
         return null;
       }
-      
+
       @Override
       public ContentSummaryComputationContext computeContentSummary(
           ContentSummaryComputationContext summary) {
@@ -115,9 +115,10 @@ public class INodeMap {
       }
       
       @Override
-      public Counts cleanSubtree(int snapshotId, int priorSnapshotId,
+      public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps,
+          int snapshotId, int priorSnapshotId,
           BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes) {
-        return null;
+          return null;
       }
 
       @Override

+ 41 - 38
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -26,6 +26,7 @@ import java.util.List;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
@@ -301,17 +302,19 @@ public abstract class INodeReference extends INode {
   }
 
   @Override // used by WithCount
-  public Quota.Counts cleanSubtree(int snapshot, int prior,
-      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
-    return referred.cleanSubtree(snapshot, prior, collectedBlocks,
+  public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, int snapshot,
+      int prior, BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes) {
+    return referred.cleanSubtree(bsps, snapshot, prior, collectedBlocks,
         removedINodes);
   }
 
   @Override // used by WithCount
   public void destroyAndCollectBlocks(
+      BlockStoragePolicySuite bsps,
       BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
     if (removeReference(this) <= 0) {
-      referred.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+      referred.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes);
     }
   }
 
@@ -322,18 +325,19 @@ public abstract class INodeReference extends INode {
   }
 
   @Override
-  public Quota.Counts computeQuotaUsage(Quota.Counts counts, boolean useCache,
-      int lastSnapshotId) {
-    return referred.computeQuotaUsage(counts, useCache, lastSnapshotId);
+  public QuotaCounts computeQuotaUsage(
+    BlockStoragePolicySuite bsps,
+    QuotaCounts counts, boolean useCache, int lastSnapshotId) {
+    return referred.computeQuotaUsage(bsps, counts, useCache, lastSnapshotId);
   }
-  
+
   @Override
   public final INodeAttributes getSnapshotINode(int snapshotId) {
     return referred.getSnapshotINode(snapshotId);
   }
 
   @Override
-  public Quota.Counts getQuotaCounts() {
+  public QuotaCounts getQuotaCounts() {
     return referred.getQuotaCounts();
   }
 
@@ -506,15 +510,15 @@ public abstract class INodeReference extends INode {
     public final ContentSummaryComputationContext computeContentSummary(
         ContentSummaryComputationContext summary) {
       //only count diskspace for WithName
-      final Quota.Counts q = Quota.Counts.newInstance();
-      computeQuotaUsage(q, false, lastSnapshotId);
-      summary.getCounts().add(Content.DISKSPACE, q.get(Quota.DISKSPACE));
+      final QuotaCounts q = new QuotaCounts.Builder().build();
+      computeQuotaUsage(summary.getBlockStoragePolicySuite(), q, false, lastSnapshotId);
+      summary.getCounts().add(Content.DISKSPACE, q.getDiskSpace());
       return summary;
     }
 
     @Override
-    public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
-        boolean useCache, int lastSnapshotId) {
+    public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
+        QuotaCounts counts, boolean useCache, int lastSnapshotId) {
       // if this.lastSnapshotId < lastSnapshotId, the rename of the referred 
       // node happened before the rename of its ancestor. This should be 
       // impossible since for WithName node we only count its children at the 
@@ -529,12 +533,12 @@ public abstract class INodeReference extends INode {
       // been updated by changes in the current tree.
       int id = lastSnapshotId != Snapshot.CURRENT_STATE_ID ? 
           lastSnapshotId : this.lastSnapshotId;
-      return referred.computeQuotaUsage(counts, false, id);
+      return referred.computeQuotaUsage(bsps, counts, false, id);
     }
     
     @Override
-    public Quota.Counts cleanSubtree(final int snapshot, int prior,
-        final BlocksMapUpdateInfo collectedBlocks,
+    public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps,
+        final int snapshot, int prior, final BlocksMapUpdateInfo collectedBlocks,
         final List<INode> removedINodes) {
       // since WithName node resides in deleted list acting as a snapshot copy,
       // the parameter snapshot must be non-null
@@ -547,16 +551,15 @@ public abstract class INodeReference extends INode {
       
       if (prior != Snapshot.NO_SNAPSHOT_ID
           && Snapshot.ID_INTEGER_COMPARATOR.compare(snapshot, prior) <= 0) {
-        return Quota.Counts.newInstance();
+        return new QuotaCounts.Builder().build();
       }
 
-      Quota.Counts counts = getReferredINode().cleanSubtree(snapshot, prior,
+      QuotaCounts counts = getReferredINode().cleanSubtree(bsps, snapshot, prior,
           collectedBlocks, removedINodes);
       INodeReference ref = getReferredINode().getParentReference();
       if (ref != null) {
         try {
-          ref.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
-              -counts.get(Quota.DISKSPACE), true);
+          ref.addSpaceConsumed(counts.negation(), true);
         } catch (QuotaExceededException e) {
           Log.warn("Should not have QuotaExceededException");
         }
@@ -567,17 +570,18 @@ public abstract class INodeReference extends INode {
         // in all the nodes existing at the time of the corresponding rename op.
         // Thus if we are deleting a snapshot before/at the snapshot associated 
         // with lastSnapshotId, we do not need to update the quota upwards.
-        counts = Quota.Counts.newInstance();
+        counts = new QuotaCounts.Builder().build();
       }
       return counts;
     }
     
     @Override
-    public void destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks,
+    public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
+        BlocksMapUpdateInfo collectedBlocks,
         final List<INode> removedINodes) {
       int snapshot = getSelfSnapshot();
       if (removeReference(this) <= 0) {
-        getReferredINode().destroyAndCollectBlocks(collectedBlocks,
+        getReferredINode().destroyAndCollectBlocks(bsps, collectedBlocks,
             removedINodes);
       } else {
         int prior = getPriorSnapshot(this);
@@ -597,12 +601,11 @@ public abstract class INodeReference extends INode {
             return;
           }
           try {
-            Quota.Counts counts = referred.cleanSubtree(snapshot, prior,
+            QuotaCounts counts = referred.cleanSubtree(bsps, snapshot, prior,
                 collectedBlocks, removedINodes);
             INodeReference ref = getReferredINode().getParentReference();
             if (ref != null) {
-              ref.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
-                  -counts.get(Quota.DISKSPACE), true);
+              ref.addSpaceConsumed(counts.negation(), true);
             }
           } catch (QuotaExceededException e) {
             LOG.error("should not exceed quota while snapshot deletion", e);
@@ -653,13 +656,13 @@ public abstract class INodeReference extends INode {
     }
     
     @Override
-    public Quota.Counts cleanSubtree(int snapshot, int prior,
+    public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, int snapshot, int prior,
         BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes) {
       if (snapshot == Snapshot.CURRENT_STATE_ID
           && prior == Snapshot.NO_SNAPSHOT_ID) {
-        Quota.Counts counts = Quota.Counts.newInstance();
-        this.computeQuotaUsage(counts, true);
-        destroyAndCollectBlocks(collectedBlocks, removedINodes);
+        QuotaCounts counts = new QuotaCounts.Builder().build();
+        this.computeQuotaUsage(bsps, counts, true);
+        destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes);
         return counts;
       } else {
         // if prior is NO_SNAPSHOT_ID, we need to check snapshot belonging to 
@@ -673,9 +676,9 @@ public abstract class INodeReference extends INode {
         if (snapshot != Snapshot.CURRENT_STATE_ID
             && prior != Snapshot.NO_SNAPSHOT_ID
             && Snapshot.ID_INTEGER_COMPARATOR.compare(snapshot, prior) <= 0) {
-          return Quota.Counts.newInstance();
+          return new QuotaCounts.Builder().build();
         }
-        return getReferredINode().cleanSubtree(snapshot, prior,
+        return getReferredINode().cleanSubtree(bsps, snapshot, prior,
             collectedBlocks, removedINodes);
       }
     }
@@ -691,10 +694,10 @@ public abstract class INodeReference extends INode {
      * WithName nodes.
      */
     @Override
-    public void destroyAndCollectBlocks(
+    public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
         BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
       if (removeReference(this) <= 0) {
-        getReferredINode().destroyAndCollectBlocks(collectedBlocks,
+        getReferredINode().destroyAndCollectBlocks(bsps, collectedBlocks,
             removedINodes);
       } else {
         // we will clean everything, including files, directories, and 
@@ -717,7 +720,7 @@ public abstract class INodeReference extends INode {
           // when calling cleanSubtree of the referred node, since we
           // compute quota usage updates before calling this destroy
           // function, we use true for countDiffChange
-          referred.cleanSubtree(snapshot, prior, collectedBlocks,
+          referred.cleanSubtree(bsps, snapshot, prior, collectedBlocks,
               removedINodes);
         } else if (referred.isDirectory()) {
           // similarly, if referred is a directory, it must be an
@@ -725,7 +728,7 @@ public abstract class INodeReference extends INode {
           INodeDirectory dir = referred.asDirectory();
           Preconditions.checkState(dir.isWithSnapshot());
           try {
-            DirectoryWithSnapshotFeature.destroyDstSubtree(dir, snapshot,
+            DirectoryWithSnapshotFeature.destroyDstSubtree(bsps, dir, snapshot,
                 prior, collectedBlocks, removedINodes);
           } catch (QuotaExceededException e) {
             LOG.error("should not exceed quota while snapshot deletion", e);
@@ -733,7 +736,7 @@ public abstract class INodeReference extends INode {
         }
       }
     }
-    
+
     private int getSelfSnapshot(final int prior) {
       WithCount wc = (WithCount) getReferredINode().asReference();
       INode referred = wc.getReferredINode();

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

@@ -23,6 +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.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 /**
@@ -72,26 +73,29 @@ public class INodeSymlink extends INodeWithAdditionalFields {
   }
   
   @Override
-  public Quota.Counts cleanSubtree(final int snapshotId, int priorSnapshotId,
+  public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps,
+      final int snapshotId, int priorSnapshotId,
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
     if (snapshotId == Snapshot.CURRENT_STATE_ID
         && priorSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
-      destroyAndCollectBlocks(collectedBlocks, removedINodes);
+      destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes);
     }
-    return Quota.Counts.newInstance(1, 0);
+    return new QuotaCounts.Builder().nameCount(1).build();
   }
   
   @Override
-  public void destroyAndCollectBlocks(final BlocksMapUpdateInfo collectedBlocks,
+  public void destroyAndCollectBlocks(final BlockStoragePolicySuite bsps,
+      final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
     removedINodes.add(this);
   }
 
   @Override
-  public Quota.Counts computeQuotaUsage(Quota.Counts counts,
-      boolean updateCache, int lastSnapshotId) {
-    counts.add(Quota.NAMESPACE, 1);
+  public QuotaCounts computeQuotaUsage(
+      BlockStoragePolicySuite bsps,
+      QuotaCounts counts, boolean useCache, int lastSnapshotId) {
+    counts.addNameSpace(1);
     return counts;
   }
 

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

@@ -71,7 +71,8 @@ public class NameNodeLayoutVersion {
     XATTRS_NAMESPACE_EXT(-59, "Increase number of xattr namespaces"),
     BLOCK_STORAGE_POLICY(-60, "Block Storage policy"),
     TRUNCATE(-61, "Truncate"),
-    APPEND_NEW_BLOCK(-62, "Support appending to new block");
+    APPEND_NEW_BLOCK(-62, "Support appending to new block"),
+    QUOTA_BY_STORAGE_TYPE(-63, "Support quota for specific storage types");
 
     private final FeatureInfo info;
 

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

@@ -68,7 +68,6 @@ import org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
-import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.AclException;
@@ -137,6 +136,7 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -1190,11 +1190,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
                        StorageType type)
       throws IOException {
     checkNNStartup();
-    if (type != null) {
-      throw new UnsupportedActionException(
-          "Quota by storage type support is not fully supported by namenode yet.");
-    }
-    namesystem.setQuota(path, namespaceQuota, diskspaceQuota);
+    namesystem.setQuota(path, namespaceQuota, diskspaceQuota, type);
   }
   
   @Override // ClientProtocol

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

@@ -39,7 +39,7 @@ public enum Quota {
     public static Counts newInstance() {
       return newInstance(0, 0);
     }
-    
+
     Counts() {
       super(Quota.class);
     }

+ 85 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/QuotaByStorageTypeEntry.java

@@ -0,0 +1,85 @@
+/**
+ * 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 com.google.common.base.Objects;
+import org.apache.hadoop.hdfs.StorageType;
+
+ public class QuotaByStorageTypeEntry {
+   private StorageType type;
+   private long quota;
+
+   public StorageType getStorageType() {
+     return type;
+   }
+
+   public long getQuota() {
+     return quota;
+   }
+
+   @Override
+   public boolean equals(Object o){
+     if (o == null) {
+       return false;
+     }
+     if (getClass() != o.getClass()) {
+       return false;
+     }
+     QuotaByStorageTypeEntry other = (QuotaByStorageTypeEntry)o;
+     return Objects.equal(type, other.type) && Objects.equal(quota, other.quota);
+   }
+
+   @Override
+   public int hashCode() {
+     return Objects.hashCode(type, quota);
+   }
+
+   @Override
+   public String toString() {
+     StringBuilder sb = new StringBuilder();
+     assert (type != null);
+     sb.append(type.toString().toLowerCase());
+     sb.append(':');
+     sb.append(quota);
+     return sb.toString();
+   }
+
+   public static class Builder {
+     private StorageType type;
+     private long quota;
+
+     public Builder setStorageType(StorageType type) {
+       this.type = type;
+       return this;
+     }
+
+     public Builder setQuota(long quota) {
+       this.quota = quota;
+       return this;
+     }
+
+     public QuotaByStorageTypeEntry build() {
+       return new QuotaByStorageTypeEntry(type, quota);
+     }
+   }
+
+   private QuotaByStorageTypeEntry(StorageType type, long quota) {
+     this.type = type;
+     this.quota = quota;
+   }
+ }

+ 179 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/QuotaCounts.java

@@ -0,0 +1,179 @@
+/**
+ * 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.StorageType;
+import org.apache.hadoop.hdfs.util.EnumCounters;
+
+/**
+ * Counters for namespace, space and storage type quota and usage.
+ */
+public class QuotaCounts {
+
+  private EnumCounters<Quota> nsSpCounts;
+  private EnumCounters<StorageType> typeCounts;
+
+  public static class Builder {
+    private EnumCounters<Quota> nsSpCounts;
+    private EnumCounters<StorageType> typeCounts;
+
+    public Builder() {
+      this.nsSpCounts = new EnumCounters<Quota>(Quota.class);
+      this.typeCounts = new EnumCounters<StorageType>(StorageType.class);
+    }
+
+    public Builder nameCount(long val) {
+      this.nsSpCounts.set(Quota.NAMESPACE, val);
+      return this;
+    }
+
+    public Builder spaceCount(long val) {
+      this.nsSpCounts.set(Quota.DISKSPACE, val);
+      return this;
+    }
+
+    public Builder typeCounts(EnumCounters<StorageType> val) {
+      if (val != null) {
+        this.typeCounts.set(val);
+      }
+      return this;
+    }
+
+    public Builder typeCounts(long val) {
+      this.typeCounts.reset(val);
+      return this;
+    }
+
+    public Builder quotaCount(QuotaCounts that) {
+      this.nsSpCounts.set(that.nsSpCounts);
+      this.typeCounts.set(that.typeCounts);
+      return this;
+    }
+
+    public QuotaCounts build() {
+      return new QuotaCounts(this);
+    }
+  }
+
+  private QuotaCounts(Builder builder) {
+    this.nsSpCounts = builder.nsSpCounts;
+    this.typeCounts = builder.typeCounts;
+  }
+
+  public void add(QuotaCounts that) {
+    this.nsSpCounts.add(that.nsSpCounts);
+    this.typeCounts.add(that.typeCounts);
+  }
+
+  public void subtract(QuotaCounts that) {
+    this.nsSpCounts.subtract(that.nsSpCounts);
+    this.typeCounts.subtract(that.typeCounts);
+  }
+
+  /**
+   * Returns a QuotaCounts whose value is {@code (-this)}.
+   *
+   * @return {@code -this}
+   */
+  public QuotaCounts negation() {
+    QuotaCounts ret = new QuotaCounts.Builder().quotaCount(this).build();
+    ret.nsSpCounts.negation();
+    ret.typeCounts.negation();
+    return ret;
+  }
+
+  public long getNameSpace(){
+    return nsSpCounts.get(Quota.NAMESPACE);
+  }
+
+  public void setNameSpace(long nameSpaceCount) {
+    this.nsSpCounts.set(Quota.NAMESPACE, nameSpaceCount);
+  }
+
+  public void addNameSpace(long nsDelta) {
+    this.nsSpCounts.add(Quota.NAMESPACE, nsDelta);
+  }
+
+  public long getDiskSpace(){
+    return nsSpCounts.get(Quota.DISKSPACE);
+  }
+
+  public void setDiskSpace(long spaceCount) {
+    this.nsSpCounts.set(Quota.DISKSPACE, spaceCount);
+  }
+
+  public void addDiskSpace(long dsDelta) {
+    this.nsSpCounts.add(Quota.DISKSPACE, dsDelta);
+  }
+
+  public EnumCounters<StorageType> getTypeSpaces() {
+    EnumCounters<StorageType> ret =
+        new EnumCounters<StorageType>(StorageType.class);
+    ret.set(typeCounts);
+    return ret;
+  }
+
+  void setTypeSpaces(EnumCounters<StorageType> that) {
+    if (that != null) {
+      this.typeCounts.set(that);
+    }
+  }
+
+  long getTypeSpace(StorageType type) {
+    return this.typeCounts.get(type);
+  }
+
+  void setTypeSpace(StorageType type, long spaceCount) {
+    this.typeCounts.set(type, spaceCount);
+  }
+
+  public void addTypeSpace(StorageType type, long delta) {
+    this.typeCounts.add(type, delta);
+  }
+
+  public void addTypeSpaces(EnumCounters<StorageType> deltas) {
+    this.typeCounts.add(deltas);
+  }
+
+  public boolean anyNsSpCountGreaterOrEqual(long val) {
+    return nsSpCounts.anyGreaterOrEqual(val);
+  }
+
+  public boolean anyTypeCountGreaterOrEqual(long val) {
+    return typeCounts.anyGreaterOrEqual(val);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    } else if (obj == null || !(obj instanceof QuotaCounts)) {
+      return false;
+    }
+    final QuotaCounts that = (QuotaCounts)obj;
+    return this.nsSpCounts.equals(that.nsSpCounts)
+        && this.typeCounts.equals(that.typeCounts);
+  }
+
+  @Override
+  public int hashCode() {
+    assert false : "hashCode not designed";
+    return 42; // any arbitrary constant will do
+  }
+}

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

@@ -21,10 +21,11 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
-import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 
 import com.google.common.base.Preconditions;
@@ -114,17 +115,21 @@ abstract class AbstractINodeDiff<N extends INode,
   }
 
   /** Combine the posterior diff and collect blocks for deletion. */
-  abstract Quota.Counts combinePosteriorAndCollectBlocks(final N currentINode,
+  abstract QuotaCounts combinePosteriorAndCollectBlocks(
+      final BlockStoragePolicySuite bsps, final N currentINode,
       final D posterior, final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes);
   
   /**
    * Delete and clear self.
+   * @param bsps The block storage policy suite used to retrieve storage policy
    * @param currentINode The inode where the deletion happens.
    * @param collectedBlocks Used to collect blocks for deletion.
+   * @param removedINodes INodes removed
    * @return quota usage delta
    */
-  abstract Quota.Counts destroyDiffAndCollectBlocks(final N currentINode,
+  abstract QuotaCounts destroyDiffAndCollectBlocks(
+      final BlockStoragePolicySuite bsps, final N currentINode,
       final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes);
 
   @Override

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

@@ -22,10 +22,11 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
-import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 
 /**
  * A list of snapshot diffs for storing snapshot data.
@@ -66,13 +67,14 @@ abstract class AbstractINodeDiffList<N extends INode,
    * @param collectedBlocks Used to collect information for blocksMap update
    * @return delta in namespace. 
    */
-  public final Quota.Counts deleteSnapshotDiff(final int snapshot,
+  public final QuotaCounts deleteSnapshotDiff(BlockStoragePolicySuite bsps,
+      final int snapshot,
       final int prior, final N currentINode,
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
     int snapshotIndex = Collections.binarySearch(diffs, snapshot);
     
-    Quota.Counts counts = Quota.Counts.newInstance();
+    QuotaCounts counts = new QuotaCounts.Builder().build();
     D removed = null;
     if (snapshotIndex == 0) {
       if (prior != Snapshot.NO_SNAPSHOT_ID) { // there is still snapshot before
@@ -80,7 +82,7 @@ abstract class AbstractINodeDiffList<N extends INode,
         diffs.get(snapshotIndex).setSnapshotId(prior);
       } else { // there is no snapshot before
         removed = diffs.remove(0);
-        counts.add(removed.destroyDiffAndCollectBlocks(currentINode,
+        counts.add(removed.destroyDiffAndCollectBlocks(bsps, currentINode,
             collectedBlocks, removedINodes));
       }
     } else if (snapshotIndex > 0) {
@@ -95,7 +97,7 @@ abstract class AbstractINodeDiffList<N extends INode,
         }
 
         counts.add(previous.combinePosteriorAndCollectBlocks(
-            currentINode, removed, collectedBlocks, removedINodes));
+            bsps, currentINode, removed, collectedBlocks, removedINodes));
         previous.setPosterior(removed.getPosterior());
         removed.setPosterior(null);
       }

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

@@ -29,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.Content;
 import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -39,7 +40,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
-import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.Time;
@@ -202,7 +203,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
    * @return The removed snapshot. Null if no snapshot with the given name
    *         exists.
    */
-  public Snapshot removeSnapshot(INodeDirectory snapshotRoot,
+  public Snapshot removeSnapshot(BlockStoragePolicySuite bsps, INodeDirectory snapshotRoot,
       String snapshotName, BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) throws SnapshotException {
     final int i = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
@@ -214,14 +215,13 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
       final Snapshot snapshot = snapshotsByNames.get(i);
       int prior = Snapshot.findLatestSnapshot(snapshotRoot, snapshot.getId());
       try {
-        Quota.Counts counts = snapshotRoot.cleanSubtree(snapshot.getId(),
+        QuotaCounts counts = snapshotRoot.cleanSubtree(bsps, snapshot.getId(),
             prior, collectedBlocks, removedINodes);
         INodeDirectory parent = snapshotRoot.getParent();
         if (parent != null) {
           // there will not be any WithName node corresponding to the deleted
           // snapshot, thus only update the quota usage in the current tree
-          parent.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
-              -counts.get(Quota.DISKSPACE), true);
+          parent.addSpaceConsumed(counts.negation(), true);
         }
       } catch(QuotaExceededException e) {
         INode.LOG.error("BUG: removeSnapshot increases namespace usage.", e);
@@ -233,6 +233,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
   }
 
   public ContentSummaryComputationContext computeContentSummary(
+      final BlockStoragePolicySuite bsps,
       final INodeDirectory snapshotRoot,
       final ContentSummaryComputationContext summary) {
     snapshotRoot.computeContentSummary(summary);

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

@@ -28,6 +28,7 @@ import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.AclStorage;
 import org.apache.hadoop.hdfs.server.namenode.Content;
 import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
@@ -38,7 +39,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
-import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 import org.apache.hadoop.hdfs.util.Diff;
 import org.apache.hadoop.hdfs.util.Diff.Container;
@@ -94,14 +95,16 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     /** clear the created list */
-    private Quota.Counts destroyCreatedList(final INodeDirectory currentINode,
+    private QuotaCounts destroyCreatedList(
+        final BlockStoragePolicySuite bsps,
+        final INodeDirectory currentINode,
         final BlocksMapUpdateInfo collectedBlocks,
         final List<INode> removedINodes) {
-      Quota.Counts counts = Quota.Counts.newInstance();
+      QuotaCounts counts = new QuotaCounts.Builder().build();
       final List<INode> createdList = getList(ListType.CREATED);
       for (INode c : createdList) {
-        c.computeQuotaUsage(counts, true);
-        c.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+        c.computeQuotaUsage(bsps, counts, true);
+        c.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes);
         // c should be contained in the children list, remove it
         currentINode.removeChild(c);
       }
@@ -110,14 +113,15 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     /** clear the deleted list */
-    private Quota.Counts destroyDeletedList(
+    private QuotaCounts destroyDeletedList(
+        final BlockStoragePolicySuite bsps,
         final BlocksMapUpdateInfo collectedBlocks,
         final List<INode> removedINodes) {
-      Quota.Counts counts = Quota.Counts.newInstance();
+      QuotaCounts counts = new QuotaCounts.Builder().build();
       final List<INode> deletedList = getList(ListType.DELETED);
       for (INode d : deletedList) {
-        d.computeQuotaUsage(counts, false);
-        d.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+        d.computeQuotaUsage(bsps, counts, false);
+        d.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes);
       }
       deletedList.clear();
       return counts;
@@ -204,18 +208,19 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     @Override
-    Quota.Counts combinePosteriorAndCollectBlocks(
+    QuotaCounts combinePosteriorAndCollectBlocks(
+        final BlockStoragePolicySuite bsps,
         final INodeDirectory currentDir, final DirectoryDiff posterior,
         final BlocksMapUpdateInfo collectedBlocks,
         final List<INode> removedINodes) {
-      final Quota.Counts counts = Quota.Counts.newInstance();
+      final QuotaCounts counts = new QuotaCounts.Builder().build();
       diff.combinePosterior(posterior.diff, new Diff.Processor<INode>() {
         /** Collect blocks for deleted files. */
         @Override
         public void process(INode inode) {
           if (inode != null) {
-            inode.computeQuotaUsage(counts, false);
-            inode.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+            inode.computeQuotaUsage(bsps, counts, false);
+            inode.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes);
           }
         }
       });
@@ -313,11 +318,12 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     @Override
-    Quota.Counts destroyDiffAndCollectBlocks(INodeDirectory currentINode,
+    QuotaCounts destroyDiffAndCollectBlocks(
+        BlockStoragePolicySuite bsps, INodeDirectory currentINode,
         BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
       // this diff has been deleted
-      Quota.Counts counts = Quota.Counts.newInstance();
-      counts.add(diff.destroyDeletedList(collectedBlocks, removedINodes));
+      QuotaCounts counts = new QuotaCounts.Builder().build();
+      counts.add(diff.destroyDeletedList(bsps, collectedBlocks, removedINodes));
       INodeDirectoryAttributes snapshotINode = getSnapshotINode();
       if (snapshotINode != null && snapshotINode.getAclFeature() != null) {
         AclStorage.removeAclFeature(snapshotINode.getAclFeature());
@@ -401,7 +407,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
   /**
    * Destroy a subtree under a DstReference node.
    */
-  public static void destroyDstSubtree(INode inode, final int snapshot,
+  public static void destroyDstSubtree(
+      final BlockStoragePolicySuite bsps, INode inode, final int snapshot,
       final int prior, final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) throws QuotaExceededException {
     Preconditions.checkArgument(prior != Snapshot.NO_SNAPSHOT_ID);
@@ -410,14 +417,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);
+        inode.cleanSubtree(bsps, snapshot, prior, collectedBlocks, removedINodes);
       } else { 
         // for DstReference node, continue this process to its subtree
-        destroyDstSubtree(inode.asReference().getReferredINode(), snapshot,
+        destroyDstSubtree(bsps, inode.asReference().getReferredINode(), snapshot,
             prior, collectedBlocks, removedINodes);
       }
     } else if (inode.isFile()) {
-      inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes);
+      inode.cleanSubtree(bsps, snapshot, prior, collectedBlocks, removedINodes);
     } else if (inode.isDirectory()) {
       Map<INode, INode> excludedNodes = null;
       INodeDirectory dir = inode.asDirectory();
@@ -431,12 +438,12 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         }
         
         if (snapshot != Snapshot.CURRENT_STATE_ID) {
-          diffList.deleteSnapshotDiff(snapshot, prior, dir, collectedBlocks,
+          diffList.deleteSnapshotDiff(bsps, snapshot, prior, dir, collectedBlocks,
               removedINodes);
         }
         priorDiff = diffList.getDiffById(prior);
         if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
-          priorDiff.diff.destroyCreatedList(dir, collectedBlocks,
+          priorDiff.diff.destroyCreatedList(bsps, dir, collectedBlocks,
               removedINodes);
         }
       }
@@ -444,7 +451,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         if (excludedNodes != null && excludedNodes.containsKey(child)) {
           continue;
         }
-        destroyDstSubtree(child, snapshot, prior, collectedBlocks,
+        destroyDstSubtree(bsps, child, snapshot, prior, collectedBlocks,
             removedINodes);
       }
     }
@@ -453,17 +460,19 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
   /**
    * Clean an inode while we move it from the deleted list of post to the
    * deleted list of prior.
+   * @param bsps The block storage policy suite.
    * @param inode The inode to clean.
    * @param post The post snapshot.
    * @param prior The id of the prior snapshot.
    * @param collectedBlocks Used to collect blocks for later deletion.
    * @return Quota usage update.
    */
-  private static Quota.Counts cleanDeletedINode(INode inode,
+  private static QuotaCounts cleanDeletedINode(
+      final BlockStoragePolicySuite bsps, INode inode,
       final int post, final int prior,
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
-    Quota.Counts counts = Quota.Counts.newInstance();
+    QuotaCounts counts = new QuotaCounts.Builder().build();
     Deque<INode> queue = new ArrayDeque<INode>();
     queue.addLast(inode);
     while (!queue.isEmpty()) {
@@ -471,13 +480,13 @@ 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);
+          wn.cleanSubtree(bsps, 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,
+        counts.add(file.getDiffs().deleteSnapshotDiff(bsps, post, prior, file,
             collectedBlocks, removedINodes));
       } else if (topNode.isDirectory()) {
         INodeDirectory dir = topNode.asDirectory();
@@ -489,7 +498,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           DirectoryDiff priorDiff = sf.getDiffs().getDiffById(prior);
           if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
             priorChildrenDiff = priorDiff.getChildrenDiff();
-            counts.add(priorChildrenDiff.destroyCreatedList(dir,
+            counts.add(priorChildrenDiff.destroyCreatedList(bsps, dir,
                 collectedBlocks, removedINodes));
           }
         }
@@ -619,27 +628,29 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     diff.diff.modify(snapshotCopy, child);
     return child;
   }
-  
-  public void clear(INodeDirectory currentINode,
+
+  public void clear(BlockStoragePolicySuite bsps, INodeDirectory currentINode,
       final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
     // destroy its diff list
     for (DirectoryDiff diff : diffs) {
-      diff.destroyDiffAndCollectBlocks(currentINode, collectedBlocks,
-          removedINodes);
+      diff.destroyDiffAndCollectBlocks(bsps, currentINode, collectedBlocks,
+        removedINodes);
     }
     diffs.clear();
   }
-  
-  public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
+
+  public QuotaCounts computeQuotaUsage4CurrentDirectory(
+    BlockStoragePolicySuite bsps, QuotaCounts counts) {
     for(DirectoryDiff d : diffs) {
       for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
-        deleted.computeQuotaUsage(counts, false, Snapshot.CURRENT_STATE_ID);
+        deleted.computeQuotaUsage(bsps, counts, false, Snapshot.CURRENT_STATE_ID);
       }
     }
     return counts;
   }
-  
-  public void computeContentSummary4Snapshot(final Content.Counts counts) {
+
+  public void computeContentSummary4Snapshot(final BlockStoragePolicySuite bsps,
+      final Content.Counts counts) {
     // Create a new blank summary context for blocking processing of subtree.
     ContentSummaryComputationContext summary = 
         new ContentSummaryComputationContext();
@@ -706,11 +717,11 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
   }
 
-  public Quota.Counts cleanDirectory(final INodeDirectory currentINode,
+  public QuotaCounts cleanDirectory(final BlockStoragePolicySuite bsps, final INodeDirectory currentINode,
       final int snapshot, int prior,
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
-    Quota.Counts counts = Quota.Counts.newInstance();
+    QuotaCounts counts = new QuotaCounts.Builder().build();
     Map<INode, INode> priorCreated = null;
     Map<INode, INode> priorDeleted = null;
     if (snapshot == Snapshot.CURRENT_STATE_ID) { // delete the current directory
@@ -718,10 +729,10 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
       // delete everything in created list
       DirectoryDiff lastDiff = diffs.getLast();
       if (lastDiff != null) {
-        counts.add(lastDiff.diff.destroyCreatedList(currentINode,
+        counts.add(lastDiff.diff.destroyCreatedList(bsps, currentINode,
             collectedBlocks, removedINodes));
       }
-      counts.add(currentINode.cleanSubtreeRecursively(snapshot, prior,
+      counts.add(currentINode.cleanSubtreeRecursively(bsps, snapshot, prior,
           collectedBlocks, removedINodes, priorDeleted));
     } else {
       // update prior
@@ -738,9 +749,9 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         }
       }
       
-      counts.add(getDiffs().deleteSnapshotDiff(snapshot, prior,
+      counts.add(getDiffs().deleteSnapshotDiff(bsps, snapshot, prior,
           currentINode, collectedBlocks, removedINodes));
-      counts.add(currentINode.cleanSubtreeRecursively(snapshot, prior,
+      counts.add(currentINode.cleanSubtreeRecursively(bsps, snapshot, prior,
           collectedBlocks, removedINodes, priorDeleted));
 
       // check priorDiff again since it may be created during the diff deletion
@@ -757,7 +768,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
             for (INode cNode : priorDiff.getChildrenDiff().getList(
                 ListType.CREATED)) {
               if (priorCreated.containsKey(cNode)) {
-                counts.add(cNode.cleanSubtree(snapshot, Snapshot.NO_SNAPSHOT_ID,
+                counts.add(cNode.cleanSubtree(bsps, snapshot, Snapshot.NO_SNAPSHOT_ID,
                     collectedBlocks, removedINodes));
               }
             }
@@ -774,7 +785,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           for (INode dNode : priorDiff.getChildrenDiff().getList(
               ListType.DELETED)) {
             if (priorDeleted == null || !priorDeleted.containsKey(dNode)) {
-              counts.add(cleanDeletedINode(dNode, snapshot, prior,
+              counts.add(cleanDeletedINode(bsps, dNode, snapshot, prior,
                   collectedBlocks, removedINodes));
             }
           }
@@ -784,7 +795,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
 
     if (currentINode.isQuotaSet()) {
       currentINode.getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
-          -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
+          counts.negation());
     }
     return counts;
   }

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

@@ -34,9 +34,11 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import com.google.common.collect.ImmutableList;
 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.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
@@ -66,12 +68,15 @@ import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
 import org.apache.hadoop.hdfs.server.namenode.INodeWithAdditionalFields;
+import org.apache.hadoop.hdfs.server.namenode.QuotaByStorageTypeEntry;
 import org.apache.hadoop.hdfs.server.namenode.SaveNamespaceContext;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.Root;
 import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
+import org.apache.hadoop.hdfs.util.EnumCounters;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
@@ -344,13 +349,31 @@ public class FSImageFormatPBSnapshot {
 
           long modTime = dirCopyInPb.getModificationTime();
           boolean noQuota = dirCopyInPb.getNsQuota() == -1
-              && dirCopyInPb.getDsQuota() == -1;
-
-          copy = noQuota ? new INodeDirectoryAttributes.SnapshotCopy(name,
-              permission, acl, modTime, xAttrs)
-              : new INodeDirectoryAttributes.CopyWithQuota(name, permission,
-                  acl, modTime, dirCopyInPb.getNsQuota(),
-                  dirCopyInPb.getDsQuota(), xAttrs);
+              && dirCopyInPb.getDsQuota() == -1
+              && (!dirCopyInPb.hasTypeQuotas());
+
+          if (noQuota) {
+            copy = new INodeDirectoryAttributes.SnapshotCopy(name,
+              permission, acl, modTime, xAttrs);
+          } else {
+            EnumCounters<StorageType> typeQuotas = null;
+            if (dirCopyInPb.hasTypeQuotas()) {
+              ImmutableList<QuotaByStorageTypeEntry> qes =
+                  FSImageFormatPBINode.Loader.loadQuotaByStorageTypeEntries(
+                      dirCopyInPb.getTypeQuotas());
+              typeQuotas = new EnumCounters<StorageType>(StorageType.class,
+                  HdfsConstants.QUOTA_RESET);
+              for (QuotaByStorageTypeEntry qe : qes) {
+                if (qe.getQuota() >= 0 && qe.getStorageType() != null &&
+                    qe.getStorageType().supportTypeQuota()) {
+                  typeQuotas.set(qe.getStorageType(), qe.getQuota());
+                }
+              }
+            }
+            copy = new INodeDirectoryAttributes.CopyWithQuota(name, permission,
+                acl, modTime, dirCopyInPb.getNsQuota(),
+                dirCopyInPb.getDsQuota(), typeQuotas, xAttrs);
+          }
         }
         // load created list
         List<INode> clist = loadCreatedList(in, dir,

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

@@ -23,12 +23,13 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
-import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 
 /**
@@ -80,13 +81,14 @@ public class FileDiff extends
   }
 
   @Override
-  Quota.Counts combinePosteriorAndCollectBlocks(INodeFile currentINode,
+  QuotaCounts combinePosteriorAndCollectBlocks(
+      BlockStoragePolicySuite bsps, INodeFile currentINode,
       FileDiff posterior, BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
     FileWithSnapshotFeature sf = currentINode.getFileWithSnapshotFeature();
     assert sf != null : "FileWithSnapshotFeature is null";
     return sf.updateQuotaAndCollectBlocks(
-        currentINode, posterior, collectedBlocks, removedINodes);
+        bsps, currentINode, posterior, collectedBlocks, removedINodes);
   }
   
   @Override
@@ -110,10 +112,10 @@ public class FileDiff extends
   }
 
   @Override
-  Quota.Counts destroyDiffAndCollectBlocks(INodeFile currentINode,
+  QuotaCounts destroyDiffAndCollectBlocks(BlockStoragePolicySuite bsps, INodeFile currentINode,
       BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
     return currentINode.getFileWithSnapshotFeature()
-        .updateQuotaAndCollectBlocks(currentINode, this, collectedBlocks,
+        .updateQuotaAndCollectBlocks(bsps, currentINode, this, collectedBlocks,
             removedINodes);
   }
 

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

@@ -21,6 +21,7 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@@ -93,7 +94,7 @@ public class FileDiffList extends
    * up to the file length of the latter.
    * Collect unused blocks of the removed snapshot.
    */
-  void combineAndCollectSnapshotBlocks(INodeFile file,
+  void combineAndCollectSnapshotBlocks(BlockStoragePolicySuite bsps, INodeFile file,
                                        FileDiff removed,
                                        BlocksMapUpdateInfo collectedBlocks,
                                        List<INode> removedINodes) {
@@ -102,7 +103,7 @@ public class FileDiffList extends
       FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
       assert sf != null : "FileWithSnapshotFeature is null";
       if(sf.isCurrentFileDeleted())
-        sf.collectBlocksAndClear(file, collectedBlocks, removedINodes);
+        sf.collectBlocksAndClear(bsps, file, collectedBlocks, removedINodes);
       return;
     }
     int p = getPrior(removed.getSnapshotId(), true);

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

@@ -21,13 +21,17 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.AclStorage;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
-import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
+import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.util.EnumCounters;
 
 /**
  * Feature for file with snapshot-related information.
@@ -114,7 +118,8 @@ public class FileWithSnapshotFeature implements INode.Feature {
     return (isCurrentFileDeleted()? "(DELETED), ": ", ") + diffs;
   }
   
-  public Quota.Counts cleanFile(final INodeFile file, final int snapshotId,
+  public QuotaCounts cleanFile(final BlockStoragePolicySuite bsps,
+      final INodeFile file, final int snapshotId,
       int priorSnapshotId, final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
@@ -123,11 +128,11 @@ public class FileWithSnapshotFeature implements INode.Feature {
         file.recordModification(priorSnapshotId);
         deleteCurrentFile();
       }
-      collectBlocksAndClear(file, collectedBlocks, removedINodes);
-      return Quota.Counts.newInstance();
+      collectBlocksAndClear(bsps, file, collectedBlocks, removedINodes);
+      return new QuotaCounts.Builder().build();
     } else { // delete the snapshot
       priorSnapshotId = getDiffs().updatePrior(snapshotId, priorSnapshotId);
-      return diffs.deleteSnapshotDiff(snapshotId, priorSnapshotId, file,
+      return diffs.deleteSnapshotDiff(bsps, snapshotId, priorSnapshotId, file,
           collectedBlocks, removedINodes);
     }
   }
@@ -136,17 +141,52 @@ public class FileWithSnapshotFeature implements INode.Feature {
     this.diffs.clear();
   }
   
-  public Quota.Counts updateQuotaAndCollectBlocks(INodeFile file,
+  public QuotaCounts updateQuotaAndCollectBlocks(BlockStoragePolicySuite bsps, INodeFile file,
       FileDiff removed, BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
     long oldDiskspace = file.diskspaceConsumed();
+
+    byte storagePolicyID = file.getStoragePolicyID();
+    BlockStoragePolicy bsp = null;
+    EnumCounters<StorageType> typeSpaces =
+        new EnumCounters<StorageType>(StorageType.class);
+    if (storagePolicyID != BlockStoragePolicySuite.ID_UNSPECIFIED) {
+      bsp = bsps.getPolicy(file.getStoragePolicyID());
+    }
+
     if (removed.snapshotINode != null) {
       short replication = removed.snapshotINode.getFileReplication();
       short currentRepl = file.getBlockReplication();
       if (currentRepl == 0) {
-        oldDiskspace = file.computeFileSize(true, true) * replication;
-      } else if (replication > currentRepl) {  
-        oldDiskspace = oldDiskspace / file.getBlockReplication() * replication;
+        long oldFileSizeNoRep = file.computeFileSize(true, true);
+        oldDiskspace =  oldFileSizeNoRep * replication;
+
+        if (bsp != null) {
+          List<StorageType> oldTypeChosen = bsp.chooseStorageTypes(replication);
+          for (StorageType t : oldTypeChosen) {
+            if (t.supportTypeQuota()) {
+              typeSpaces.add(t, -oldFileSizeNoRep);
+            }
+          }
+        }
+      } else if (replication > currentRepl) {
+        long oldFileSizeNoRep = file.diskspaceConsumedNoReplication();
+        oldDiskspace = oldFileSizeNoRep * replication;
+
+        if (bsp != null) {
+          List<StorageType> oldTypeChosen = bsp.chooseStorageTypes(replication);
+          for (StorageType t : oldTypeChosen) {
+            if (t.supportTypeQuota()) {
+              typeSpaces.add(t, -oldFileSizeNoRep);
+            }
+          }
+          List<StorageType> newTypeChosen = bsp.chooseStorageTypes(currentRepl);
+          for (StorageType t: newTypeChosen) {
+            if (t.supportTypeQuota()) {
+              typeSpaces.add(t, oldFileSizeNoRep);
+            }
+          }
+        }
       }
       AclFeature aclFeature = removed.getSnapshotINode().getAclFeature();
       if (aclFeature != null) {
@@ -155,21 +195,24 @@ public class FileWithSnapshotFeature implements INode.Feature {
     }
 
     getDiffs().combineAndCollectSnapshotBlocks(
-        file, removed, collectedBlocks, removedINodes);
+        bsps, file, removed, collectedBlocks, removedINodes);
 
     long dsDelta = oldDiskspace - file.diskspaceConsumed();
-    return Quota.Counts.newInstance(0, dsDelta);
+    return new QuotaCounts.Builder().
+        spaceCount(dsDelta).
+        typeCounts(typeSpaces).
+        build();
   }
 
   /**
    * If some blocks at the end of the block list no longer belongs to
    * any inode, collect them and update the block list.
    */
-  public void collectBlocksAndClear(final INodeFile file,
+  public void collectBlocksAndClear(final BlockStoragePolicySuite bsps, final INodeFile file,
       final BlocksMapUpdateInfo info, final List<INode> removedINodes) {
     // check if everything is deleted.
     if (isCurrentFileDeleted() && getDiffs().asList().isEmpty()) {
-      file.destroyAndCollectBlocks(info, removedINodes);
+      file.destroyAndCollectBlocks(bsps, info, removedINodes);
       return;
     }
     // find max file size.

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

@@ -234,7 +234,8 @@ public class SnapshotManager implements SnapshotStatsMXBean {
       BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
       throws IOException {
     INodeDirectory srcRoot = getSnapshottableRoot(iip);
-    srcRoot.removeSnapshot(snapshotName, collectedBlocks, removedINodes);
+    srcRoot.removeSnapshot(fsdir.getBlockStoragePolicySuite(), snapshotName,
+        collectedBlocks, removedINodes);
     numSnapshots.getAndDecrement();
   }
 

+ 33 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/EnumCounters.java

@@ -50,6 +50,14 @@ public class EnumCounters<E extends Enum<E>> {
     this.enumClass = enumClass;
     this.counters = new long[enumConstants.length];
   }
+
+  public EnumCounters(final Class<E> enumClass, long defaultVal) {
+    final E[] enumConstants = enumClass.getEnumConstants();
+    Preconditions.checkNotNull(enumConstants);
+    this.enumClass = enumClass;
+    this.counters = new long[enumConstants.length];
+    reset(defaultVal);
+  }
   
   /** @return the value of counter e. */
   public final long get(final E e) {
@@ -77,9 +85,7 @@ public class EnumCounters<E extends Enum<E>> {
 
   /** Reset all counters to zero. */
   public final void reset() {
-    for(int i = 0; i < counters.length; i++) {
-      this.counters[i] = 0L;
-    }
+    reset(0L);
   }
 
   /** Add the given value to counter e. */
@@ -143,6 +149,30 @@ public class EnumCounters<E extends Enum<E>> {
     return b.substring(0, b.length() - 2);
   }
 
+  public final void reset(long val) {
+    for(int i = 0; i < counters.length; i++) {
+      this.counters[i] = val;
+    }
+  }
+
+  public boolean allLessOrEqual(long val) {
+    for (long c : counters) {
+      if (c > val) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  public boolean anyGreaterOrEqual(long val) {
+    for (long c: counters) {
+      if (c >= val) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   /**
    * A factory for creating counters.
    * 

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto

@@ -141,6 +141,15 @@ message INodeSection {
     optional uint32 storagePolicyID = 10;
   }
 
+  message QuotaByStorageTypeEntryProto {
+    required StorageTypeProto storageType = 1;
+    required uint64 quota = 2;
+  }
+
+  message QuotaByStorageTypeFeatureProto {
+    repeated QuotaByStorageTypeEntryProto quotas = 1;
+  }
+
   message INodeDirectory {
     optional uint64 modificationTime = 1;
     // namespace quota
@@ -150,6 +159,7 @@ message INodeSection {
     optional fixed64 permission = 4;
     optional AclFeatureProto acl = 5;
     optional XAttrFeatureProto xAttrs = 6;
+    optional QuotaByStorageTypeFeatureProto typeQuotas = 7;
   }
 
   message INodeSymlink {

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -1225,6 +1225,8 @@ public class DFSTestUtil {
     // OP_SET_QUOTA 14
     filesystem.setQuota(pathDirectoryMkdir, 1000L, 
         HdfsConstants.QUOTA_DONT_SET);
+    // OP_SET_QUOTA_BY_STORAGETYPE
+    filesystem.setQuotaByStorageType(pathDirectoryMkdir, StorageType.SSD, 888L);
     // OP_RENAME 15
     fc.rename(pathFileCreate, pathFileMoved, Rename.NONE);
     // OP_CONCAT_DELETE 16

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java

@@ -72,7 +72,7 @@ public class TestDFSInotifyEventInputStream {
    */
   @Test
   public void testOpcodeCount() {
-    Assert.assertEquals(49, FSEditLogOpCodes.values().length);
+    Assert.assertEquals(50, FSEditLogOpCodes.values().length);
   }
 
 

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java

@@ -549,6 +549,13 @@ public abstract class FSImageTestUtil {
   public static FSImage getFSImage(NameNode node) {
     return node.getFSImage();
   }
+
+  /**
+   * get NameSpace quota.
+   */
+  public static long getNSQuota(FSNamesystem ns) {
+    return ns.dir.rootDir.getQuotaCounts().getNameSpace();
+  }
   
   public static void assertNNFilesMatch(MiniDFSCluster cluster) throws Exception {
     List<File> curDirs = Lists.newArrayList();

+ 17 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java

@@ -81,10 +81,10 @@ public class TestDiskspaceQuotaUpdate {
     INode fnode = fsdir.getINode4Write(foo.toString());
     assertTrue(fnode.isDirectory());
     assertTrue(fnode.isQuotaSet());
-    Quota.Counts cnt = fnode.asDirectory().getDirectoryWithQuotaFeature()
+    QuotaCounts cnt = fnode.asDirectory().getDirectoryWithQuotaFeature()
         .getSpaceConsumed();
-    assertEquals(2, cnt.get(Quota.NAMESPACE));
-    assertEquals(fileLen * REPLICATION, cnt.get(Quota.DISKSPACE));
+    assertEquals(2, cnt.getNameSpace());
+    assertEquals(fileLen * REPLICATION, cnt.getDiskSpace());
   }
 
   /**
@@ -105,10 +105,10 @@ public class TestDiskspaceQuotaUpdate {
 
     INodeDirectory fooNode = fsdir.getINode4Write(foo.toString()).asDirectory();
     assertTrue(fooNode.isQuotaSet());
-    Quota.Counts quota = fooNode.getDirectoryWithQuotaFeature()
+    QuotaCounts quota = fooNode.getDirectoryWithQuotaFeature()
         .getSpaceConsumed();
-    long ns = quota.get(Quota.NAMESPACE);
-    long ds = quota.get(Quota.DISKSPACE);
+    long ns = quota.getNameSpace();
+    long ds = quota.getDiskSpace();
     assertEquals(2, ns); // foo and bar
     assertEquals(currentFileLen * REPLICATION, ds);
     ContentSummary c = dfs.getContentSummary(foo);
@@ -119,8 +119,8 @@ public class TestDiskspaceQuotaUpdate {
     currentFileLen += BLOCKSIZE;
 
     quota = fooNode.getDirectoryWithQuotaFeature().getSpaceConsumed();
-    ns = quota.get(Quota.NAMESPACE);
-    ds = quota.get(Quota.DISKSPACE);
+    ns = quota.getNameSpace();
+    ds = quota.getDiskSpace();
     assertEquals(2, ns); // foo and bar
     assertEquals(currentFileLen * REPLICATION, ds);
     c = dfs.getContentSummary(foo);
@@ -131,8 +131,8 @@ public class TestDiskspaceQuotaUpdate {
     currentFileLen += (BLOCKSIZE * 3 + BLOCKSIZE / 8);
 
     quota = fooNode.getDirectoryWithQuotaFeature().getSpaceConsumed();
-    ns = quota.get(Quota.NAMESPACE);
-    ds = quota.get(Quota.DISKSPACE);
+    ns = quota.getNameSpace();
+    ds = quota.getDiskSpace();
     assertEquals(2, ns); // foo and bar
     assertEquals(currentFileLen * REPLICATION, ds);
     c = dfs.getContentSummary(foo);
@@ -156,10 +156,10 @@ public class TestDiskspaceQuotaUpdate {
         .of(HdfsDataOutputStream.SyncFlag.UPDATE_LENGTH));
 
     INodeDirectory fooNode = fsdir.getINode4Write(foo.toString()).asDirectory();
-    Quota.Counts quota = fooNode.getDirectoryWithQuotaFeature()
+    QuotaCounts quota = fooNode.getDirectoryWithQuotaFeature()
         .getSpaceConsumed();
-    long ns = quota.get(Quota.NAMESPACE);
-    long ds = quota.get(Quota.DISKSPACE);
+    long ns = quota.getNameSpace();
+    long ds = quota.getDiskSpace();
     assertEquals(2, ns); // foo and bar
     assertEquals(BLOCKSIZE * 2 * REPLICATION, ds); // file is under construction
 
@@ -168,8 +168,8 @@ public class TestDiskspaceQuotaUpdate {
 
     fooNode = fsdir.getINode4Write(foo.toString()).asDirectory();
     quota = fooNode.getDirectoryWithQuotaFeature().getSpaceConsumed();
-    ns = quota.get(Quota.NAMESPACE);
-    ds = quota.get(Quota.DISKSPACE);
+    ns = quota.getNameSpace();
+    ds = quota.getDiskSpace();
     assertEquals(2, ns);
     assertEquals((BLOCKSIZE + BLOCKSIZE / 2) * REPLICATION, ds);
 
@@ -177,8 +177,8 @@ public class TestDiskspaceQuotaUpdate {
     DFSTestUtil.appendFile(dfs, bar, BLOCKSIZE);
 
     quota = fooNode.getDirectoryWithQuotaFeature().getSpaceConsumed();
-    ns = quota.get(Quota.NAMESPACE);
-    ds = quota.get(Quota.DISKSPACE);
+    ns = quota.getNameSpace();
+    ds = quota.getDiskSpace();
     assertEquals(2, ns); // foo and bar
     assertEquals((BLOCKSIZE * 2 + BLOCKSIZE / 2) * REPLICATION, ds);
   }

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

@@ -158,7 +158,7 @@ public class TestFSImageWithSnapshot {
     fsn.getFSDirectory().writeLock();
     try {
       loader.load(imageFile, false);
-      FSImage.updateCountForQuota(
+      FSImage.updateCountForQuota(fsn.getBlockManager().getStoragePolicySuite(),
           INodeDirectory.valueOf(fsn.getFSDirectory().getINode("/"), "/"));
     } finally {
       fsn.getFSDirectory().writeUnlock();

+ 524 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaByStorageType.java

@@ -0,0 +1,524 @@
+/**
+ * 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 static org.apache.hadoop.hdfs.StorageType.DEFAULT;
+  import static org.junit.Assert.assertEquals;
+  import static org.junit.Assert.assertFalse;
+  import static org.junit.Assert.assertTrue;
+  import static org.junit.Assert.fail;
+
+  import org.apache.commons.logging.Log;
+  import org.apache.commons.logging.LogFactory;
+  import org.apache.hadoop.conf.Configuration;
+  import org.apache.hadoop.fs.Path;
+  import org.apache.hadoop.hdfs.*;
+  import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+  import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
+  import org.junit.After;
+  import org.junit.Before;
+  import org.junit.Test;
+
+public class TestQuotaByStorageType {
+
+  private static final int BLOCKSIZE = 1024;
+  private static final short REPLICATION = 3;
+  static final long seed = 0L;
+  private static final Path dir = new Path("/TestQuotaByStorageType");
+
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private FSDirectory fsdir;
+  private DistributedFileSystem dfs;
+  private FSNamesystem fsn;
+
+  protected static final Log LOG = LogFactory.getLog(TestQuotaByStorageType.class);
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
+
+    // Setup a 3-node cluster and configure
+    // each node with 1 SSD and 1 DISK without capacity limitation
+    cluster = new MiniDFSCluster
+        .Builder(conf)
+        .numDataNodes(REPLICATION)
+        .storageTypes(new StorageType[]{StorageType.SSD, DEFAULT})
+        .build();
+    cluster.waitActive();
+
+    fsdir = cluster.getNamesystem().getFSDirectory();
+    dfs = cluster.getFileSystem();
+    fsn = cluster.getNamesystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeWithFileCreateOneSSD() throws Exception {
+    testQuotaByStorageTypeWithFileCreateCase(
+        HdfsConstants.ONESSD_STORAGE_POLICY_NAME,
+        StorageType.SSD,
+        (short)1);
+  }
+
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeWithFileCreateAllSSD() throws Exception {
+    testQuotaByStorageTypeWithFileCreateCase(
+        HdfsConstants.ALLSSD_STORAGE_POLICY_NAME,
+        StorageType.SSD,
+        (short)3);
+  }
+
+  void testQuotaByStorageTypeWithFileCreateCase(
+      String storagePolicy, StorageType storageType, short replication) throws Exception {
+    final Path foo = new Path(dir, "foo");
+    Path createdFile1 = new Path(foo, "created_file1.data");
+    dfs.mkdirs(foo);
+
+    // set storage policy on directory "foo" to storagePolicy
+    dfs.setStoragePolicy(foo, storagePolicy);
+
+    // set quota by storage type on directory "foo"
+    dfs.setQuotaByStorageType(foo, storageType, BLOCKSIZE * 10);
+
+    INode fnode = fsdir.getINode4Write(foo.toString());
+    assertTrue(fnode.isDirectory());
+    assertTrue(fnode.isQuotaSet());
+
+    // Create file of size 2 * BLOCKSIZE under directory "foo"
+    long file1Len = BLOCKSIZE * 2 + BLOCKSIZE / 2;
+    int bufLen = BLOCKSIZE / 16;
+    DFSTestUtil.createFile(dfs, createdFile1, bufLen, file1Len, BLOCKSIZE, REPLICATION, seed);
+
+    // Verify space consumed and remaining quota
+    long storageTypeConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(storageType);
+    assertEquals(file1Len * replication, storageTypeConsumed);
+  }
+
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeWithFileCreateAppend() throws Exception {
+    final Path foo = new Path(dir, "foo");
+    Path createdFile1 = new Path(foo, "created_file1.data");
+    dfs.mkdirs(foo);
+
+    // set storage policy on directory "foo" to ONESSD
+    dfs.setStoragePolicy(foo, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+
+    // set quota by storage type on directory "foo"
+    dfs.setQuotaByStorageType(foo, StorageType.SSD, BLOCKSIZE * 4);
+    INode fnode = fsdir.getINode4Write(foo.toString());
+    assertTrue(fnode.isDirectory());
+    assertTrue(fnode.isQuotaSet());
+
+    // Create file of size 2 * BLOCKSIZE under directory "foo"
+    long file1Len = BLOCKSIZE * 2;
+    int bufLen = BLOCKSIZE / 16;
+    DFSTestUtil.createFile(dfs, createdFile1, bufLen, file1Len, BLOCKSIZE, REPLICATION, seed);
+
+    // Verify space consumed and remaining quota
+    long ssdConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    assertEquals(file1Len, ssdConsumed);
+
+    // append several blocks
+    int appendLen = BLOCKSIZE * 2;
+    DFSTestUtil.appendFile(dfs, createdFile1, appendLen);
+    file1Len += appendLen;
+
+    ssdConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    assertEquals(file1Len, ssdConsumed);
+  }
+
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeWithFileCreateDelete() throws Exception {
+    final Path foo = new Path(dir, "foo");
+    Path createdFile1 = new Path(foo, "created_file1.data");
+    dfs.mkdirs(foo);
+
+    dfs.setStoragePolicy(foo, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+
+    // set quota by storage type on directory "foo"
+    dfs.setQuotaByStorageType(foo, StorageType.SSD, BLOCKSIZE * 10);
+    INode fnode = fsdir.getINode4Write(foo.toString());
+    assertTrue(fnode.isDirectory());
+    assertTrue(fnode.isQuotaSet());
+
+    // Create file of size 2.5 * BLOCKSIZE under directory "foo"
+    long file1Len = BLOCKSIZE * 2 + BLOCKSIZE / 2;
+    int bufLen = BLOCKSIZE / 16;
+    DFSTestUtil.createFile(dfs, createdFile1, bufLen, file1Len, BLOCKSIZE, REPLICATION, seed);
+
+    // Verify space consumed and remaining quota
+    long storageTypeConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    assertEquals(file1Len, storageTypeConsumed);
+
+    // Delete file and verify the consumed space of the storage type is updated
+    dfs.delete(createdFile1, false);
+    storageTypeConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    assertEquals(0, storageTypeConsumed);
+
+    QuotaCounts counts = new QuotaCounts.Builder().build();
+    fnode.computeQuotaUsage(fsn.getBlockManager().getStoragePolicySuite(), counts, true);
+    assertEquals(fnode.dumpTreeRecursively().toString(), 0,
+        counts.getTypeSpaces().get(StorageType.SSD));
+  }
+
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeWithFileCreateRename() throws Exception {
+    final Path foo = new Path(dir, "foo");
+    dfs.mkdirs(foo);
+    Path createdFile1foo = new Path(foo, "created_file1.data");
+
+    final Path bar = new Path(dir, "bar");
+    dfs.mkdirs(bar);
+    Path createdFile1bar = new Path(bar, "created_file1.data");
+
+    // set storage policy on directory "foo" and "bar" to ONESSD
+    dfs.setStoragePolicy(foo, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+    dfs.setStoragePolicy(bar, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+
+    // set quota by storage type on directory "foo"
+    dfs.setQuotaByStorageType(foo, StorageType.SSD, BLOCKSIZE * 4);
+    dfs.setQuotaByStorageType(bar, StorageType.SSD, BLOCKSIZE * 2);
+
+    INode fnode = fsdir.getINode4Write(foo.toString());
+    assertTrue(fnode.isDirectory());
+    assertTrue(fnode.isQuotaSet());
+
+    // Create file of size 3 * BLOCKSIZE under directory "foo"
+    long file1Len = BLOCKSIZE * 3;
+    int bufLen = BLOCKSIZE / 16;
+    DFSTestUtil.createFile(dfs, createdFile1foo, bufLen, file1Len, BLOCKSIZE, REPLICATION, seed);
+
+    // Verify space consumed and remaining quota
+    long ssdConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    ;
+    assertEquals(file1Len, ssdConsumed);
+
+    // move file from foo to bar
+    try {
+      dfs.rename(createdFile1foo, createdFile1bar);
+      fail("Should have failed with QuotaByStorageTypeExceededException ");
+    } catch (Throwable t) {
+      LOG.info("Got expected exception ", t);
+    }
+  }
+
+  /**
+   * Test if the quota can be correctly updated for create file even
+   * QuotaByStorageTypeExceededException is thrown
+   */
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeExceptionWithFileCreate() throws Exception {
+    final Path foo = new Path(dir, "foo");
+    Path createdFile1 = new Path(foo, "created_file1.data");
+    dfs.mkdirs(foo);
+
+    dfs.setStoragePolicy(foo, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+    dfs.setQuotaByStorageType(foo, StorageType.SSD, BLOCKSIZE * 4);
+
+    INode fnode = fsdir.getINode4Write(foo.toString());
+    assertTrue(fnode.isDirectory());
+    assertTrue(fnode.isQuotaSet());
+
+    // Create the 1st file of size 2 * BLOCKSIZE under directory "foo" and expect no exception
+    long file1Len = BLOCKSIZE * 2;
+    int bufLen = BLOCKSIZE / 16;
+    DFSTestUtil.createFile(dfs, createdFile1, bufLen, file1Len, BLOCKSIZE, REPLICATION, seed);
+    long currentSSDConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    assertEquals(file1Len, currentSSDConsumed);
+
+    // Create the 2nd file of size 1.5 * BLOCKSIZE under directory "foo" and expect no exception
+    Path createdFile2 = new Path(foo, "created_file2.data");
+    long file2Len = BLOCKSIZE + BLOCKSIZE / 2;
+    DFSTestUtil.createFile(dfs, createdFile2, bufLen, file2Len, BLOCKSIZE, REPLICATION, seed);
+    currentSSDConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+
+    assertEquals(file1Len + file2Len, currentSSDConsumed);
+
+    // Create the 3rd file of size BLOCKSIZE under directory "foo" and expect quota exceeded exception
+    Path createdFile3 = new Path(foo, "created_file3.data");
+    long file3Len = BLOCKSIZE;
+
+    try {
+      DFSTestUtil.createFile(dfs, createdFile3, bufLen, file3Len, BLOCKSIZE, REPLICATION, seed);
+      fail("Should have failed with QuotaByStorageTypeExceededException ");
+    } catch (Throwable t) {
+      LOG.info("Got expected exception ", t);
+
+      currentSSDConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+          .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+      assertEquals(file1Len + file2Len, currentSSDConsumed);
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeParentOffChildOff() throws Exception {
+    final Path parent = new Path(dir, "parent");
+    final Path child = new Path(parent, "child");
+    dfs.mkdirs(parent);
+    dfs.mkdirs(child);
+
+    dfs.setStoragePolicy(parent, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+
+    // Create file of size 2.5 * BLOCKSIZE under child directory.
+    // Since both parent and child directory do not have SSD quota set,
+    // expect succeed without exception
+    Path createdFile1 = new Path(child, "created_file1.data");
+    long file1Len = BLOCKSIZE * 2 + BLOCKSIZE / 2;
+    int bufLen = BLOCKSIZE / 16;
+    DFSTestUtil.createFile(dfs, createdFile1, bufLen, file1Len, BLOCKSIZE,
+        REPLICATION, seed);
+
+    // Verify SSD usage at the root level as both parent/child don't have DirectoryWithQuotaFeature
+    INode fnode = fsdir.getINode4Write("/");
+    long ssdConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    assertEquals(file1Len, ssdConsumed);
+
+  }
+
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeParentOffChildOn() throws Exception {
+    final Path parent = new Path(dir, "parent");
+    final Path child = new Path(parent, "child");
+    dfs.mkdirs(parent);
+    dfs.mkdirs(child);
+
+    dfs.setStoragePolicy(parent, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+    dfs.setQuotaByStorageType(child, StorageType.SSD, 2 * BLOCKSIZE);
+
+    // Create file of size 2.5 * BLOCKSIZE under child directory
+    // Since child directory have SSD quota of 2 * BLOCKSIZE,
+    // expect an exception when creating files under child directory.
+    Path createdFile1 = new Path(child, "created_file1.data");
+    long file1Len = BLOCKSIZE * 2 + BLOCKSIZE / 2;
+    int bufLen = BLOCKSIZE / 16;
+    try {
+      DFSTestUtil.createFile(dfs, createdFile1, bufLen, file1Len, BLOCKSIZE,
+          REPLICATION, seed);
+      fail("Should have failed with QuotaByStorageTypeExceededException ");
+    } catch (Throwable t) {
+      LOG.info("Got expected exception ", t);
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeParentOnChildOff() throws Exception {
+    short replication = 1;
+    final Path parent = new Path(dir, "parent");
+    final Path child = new Path(parent, "child");
+    dfs.mkdirs(parent);
+    dfs.mkdirs(child);
+
+    dfs.setStoragePolicy(parent, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+    dfs.setQuotaByStorageType(parent, StorageType.SSD, 3 * BLOCKSIZE);
+
+    // Create file of size 2.5 * BLOCKSIZE under child directory
+    // Verify parent Quota applies
+    Path createdFile1 = new Path(child, "created_file1.data");
+    long file1Len = BLOCKSIZE * 2 + BLOCKSIZE / 2;
+    int bufLen = BLOCKSIZE / 16;
+    DFSTestUtil.createFile(dfs, createdFile1, bufLen, file1Len, BLOCKSIZE,
+        replication, seed);
+
+    INode fnode = fsdir.getINode4Write(parent.toString());
+    assertTrue(fnode.isDirectory());
+    assertTrue(fnode.isQuotaSet());
+    long currentSSDConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    ;
+    assertEquals(file1Len, currentSSDConsumed);
+
+    // Create the 2nd file of size BLOCKSIZE under child directory and expect quota exceeded exception
+    Path createdFile2 = new Path(child, "created_file2.data");
+    long file2Len = BLOCKSIZE;
+
+    try {
+      DFSTestUtil.createFile(dfs, createdFile2, bufLen, file2Len, BLOCKSIZE, replication, seed);
+      fail("Should have failed with QuotaByStorageTypeExceededException ");
+    } catch (Throwable t) {
+      LOG.info("Got expected exception ", t);
+      currentSSDConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+          .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+      assertEquals(file1Len, currentSSDConsumed);
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeParentOnChildOn() throws Exception {
+    final Path parent = new Path(dir, "parent");
+    final Path child = new Path(parent, "child");
+    dfs.mkdirs(parent);
+    dfs.mkdirs(child);
+
+    dfs.setStoragePolicy(parent, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+    dfs.setQuotaByStorageType(parent, StorageType.SSD, 2 * BLOCKSIZE);
+    dfs.setQuotaByStorageType(child, StorageType.SSD, 3 * BLOCKSIZE);
+
+    // Create file of size 2.5 * BLOCKSIZE under child directory
+    // Verify parent Quota applies
+    Path createdFile1 = new Path(child, "created_file1.data");
+    long file1Len = BLOCKSIZE * 2 + BLOCKSIZE / 2;
+    int bufLen = BLOCKSIZE / 16;
+    try {
+      DFSTestUtil.createFile(dfs, createdFile1, bufLen, file1Len, BLOCKSIZE,
+          REPLICATION, seed);
+      fail("Should have failed with QuotaByStorageTypeExceededException ");
+    } catch (Throwable t) {
+      LOG.info("Got expected exception ", t);
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeWithTraditionalQuota() throws Exception {
+    final Path foo = new Path(dir, "foo");
+    dfs.mkdirs(foo);
+    dfs.setQuota(foo, Long.MAX_VALUE - 1, REPLICATION * BLOCKSIZE * 10);
+    INode fnode = fsdir.getINode4Write(foo.toString());
+    assertTrue(fnode.isDirectory());
+    assertTrue(fnode.isQuotaSet());
+
+    Path createdFile = new Path(foo, "created_file.data");
+    long fileLen = BLOCKSIZE * 2 + BLOCKSIZE / 2;
+    DFSTestUtil.createFile(dfs, createdFile, BLOCKSIZE / 16,
+        fileLen, BLOCKSIZE, REPLICATION, seed);
+
+    QuotaCounts cnt = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed();
+    assertEquals(2, cnt.getNameSpace());
+    assertEquals(fileLen * REPLICATION, cnt.getDiskSpace());
+
+    dfs.delete(createdFile, true);
+
+    QuotaCounts cntAfterDelete = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed();
+    assertEquals(1, cntAfterDelete.getNameSpace());
+    assertEquals(0, cntAfterDelete.getDiskSpace());
+
+    // Validate the computeQuotaUsage()
+    QuotaCounts counts = new QuotaCounts.Builder().build();
+    fnode.computeQuotaUsage(fsn.getBlockManager().getStoragePolicySuite(), counts, true);
+    assertEquals(fnode.dumpTreeRecursively().toString(), 1,
+        counts.getNameSpace());
+    assertEquals(fnode.dumpTreeRecursively().toString(), 0,
+        counts.getDiskSpace());
+  }
+
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeWithSnapshot() throws Exception {
+    final Path sub1 = new Path(dir, "Sub1");
+    dfs.mkdirs(sub1);
+
+    // Setup ONE_SSD policy and SSD quota of 4 * BLOCKSIZE on sub1
+    dfs.setStoragePolicy(sub1, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+    dfs.setQuotaByStorageType(sub1, StorageType.SSD, 4 * BLOCKSIZE);
+
+    INode sub1Node = fsdir.getINode4Write(sub1.toString());
+    assertTrue(sub1Node.isDirectory());
+    assertTrue(sub1Node.isQuotaSet());
+
+    // Create file1 of size 2 * BLOCKSIZE under sub1
+    Path file1 = new Path(sub1, "file1");
+    long file1Len = 2 * BLOCKSIZE;
+    DFSTestUtil.createFile(dfs, file1, file1Len, REPLICATION, seed);
+
+    // Create snapshot on sub1 named s1
+    SnapshotTestHelper.createSnapshot(dfs, sub1, "s1");
+
+    // Verify sub1 SSD usage is unchanged after creating snapshot s1
+    long ssdConsumed = sub1Node.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    assertEquals(file1Len, ssdConsumed);
+
+    // Delete file1
+    dfs.delete(file1, false);
+
+    // Verify sub1 SSD usage is unchanged due to the existence of snapshot s1
+    ssdConsumed = sub1Node.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    assertEquals(file1Len, ssdConsumed);
+
+    QuotaCounts counts1 = new QuotaCounts.Builder().build();
+    sub1Node.computeQuotaUsage(fsn.getBlockManager().getStoragePolicySuite(), counts1, true);
+    assertEquals(sub1Node.dumpTreeRecursively().toString(), file1Len,
+        counts1.getTypeSpaces().get(StorageType.SSD));
+
+    // Delete the snapshot s1
+    dfs.deleteSnapshot(sub1, "s1");
+
+    // Verify sub1 SSD usage is fully reclaimed and changed to 0
+    ssdConsumed = sub1Node.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    assertEquals(0, ssdConsumed);
+
+    QuotaCounts counts2 = new QuotaCounts.Builder().build();
+    sub1Node.computeQuotaUsage(fsn.getBlockManager().getStoragePolicySuite(), counts2, true);
+    assertEquals(sub1Node.dumpTreeRecursively().toString(), 0,
+        counts2.getTypeSpaces().get(StorageType.SSD));
+  }
+
+  @Test(timeout = 60000)
+  public void testQuotaByStorageTypeWithFileCreateTruncate() throws Exception {
+    final Path foo = new Path(dir, "foo");
+    Path createdFile1 = new Path(foo, "created_file1.data");
+    dfs.mkdirs(foo);
+
+    // set storage policy on directory "foo" to ONESSD
+    dfs.setStoragePolicy(foo, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+
+    // set quota by storage type on directory "foo"
+    dfs.setQuotaByStorageType(foo, StorageType.SSD, BLOCKSIZE * 4);
+    INode fnode = fsdir.getINode4Write(foo.toString());
+    assertTrue(fnode.isDirectory());
+    assertTrue(fnode.isQuotaSet());
+
+    // Create file of size 2 * BLOCKSIZE under directory "foo"
+    long file1Len = BLOCKSIZE * 2;
+    int bufLen = BLOCKSIZE / 16;
+    DFSTestUtil.createFile(dfs, createdFile1, bufLen, file1Len, BLOCKSIZE, REPLICATION, seed);
+
+    // Verify SSD consumed before truncate
+    long ssdConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    assertEquals(file1Len, ssdConsumed);
+
+    // Truncate file to 1 * BLOCKSIZE
+    int newFile1Len = BLOCKSIZE * 1;
+    dfs.truncate(createdFile1, newFile1Len);
+
+    // Verify SSD consumed after truncate
+    ssdConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
+        .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
+    assertEquals(newFile1Len, ssdConsumed);
+  }
+}

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

@@ -63,7 +63,7 @@ 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.QuotaCounts;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
@@ -1198,15 +1198,15 @@ public class TestRenameWithSnapshots {
     assertFalse(hdfs.exists(bar_s2));
     restartClusterAndCheckImage(true);
     // make sure the whole referred subtree has been destroyed
-    Quota.Counts q = fsdir.getRoot().getDirectoryWithQuotaFeature().getSpaceConsumed();  
-    assertEquals(3, q.get(Quota.NAMESPACE));
-    assertEquals(0, q.get(Quota.DISKSPACE));
+    QuotaCounts q = fsdir.getRoot().getDirectoryWithQuotaFeature().getSpaceConsumed();
+    assertEquals(3, q.getNameSpace());
+    assertEquals(0, q.getDiskSpace());
     
     hdfs.deleteSnapshot(sdir1, "s1");
     restartClusterAndCheckImage(true);
-    q = fsdir.getRoot().getDirectoryWithQuotaFeature().getSpaceConsumed();  
-    assertEquals(3, q.get(Quota.NAMESPACE));
-    assertEquals(0, q.get(Quota.DISKSPACE));
+    q = fsdir.getRoot().getDirectoryWithQuotaFeature().getSpaceConsumed();
+    assertEquals(3, q.getNameSpace());
+    assertEquals(0, q.getDiskSpace());
   }
   
   /**
@@ -1600,9 +1600,9 @@ public class TestRenameWithSnapshots {
     // check dir2
     INodeDirectory dir2Node = fsdir2.getINode4Write(dir2.toString()).asDirectory();
     assertTrue(dir2Node.isSnapshottable());
-    Quota.Counts counts = dir2Node.computeQuotaUsage();
-    assertEquals(2, counts.get(Quota.NAMESPACE));
-    assertEquals(0, counts.get(Quota.DISKSPACE));
+    QuotaCounts counts = dir2Node.computeQuotaUsage(fsdir.getBlockStoragePolicySuite());
+    assertEquals(2, counts.getNameSpace());
+    assertEquals(0, counts.getDiskSpace());
     childrenList = ReadOnlyList.Util.asList(dir2Node.asDirectory()
         .getChildrenList(Snapshot.CURRENT_STATE_ID));
     assertEquals(1, childrenList.size());
@@ -1674,9 +1674,9 @@ public class TestRenameWithSnapshots {
     // check dir2
     INodeDirectory dir2Node = fsdir2.getINode4Write(dir2.toString()).asDirectory();
     assertTrue(dir2Node.isSnapshottable());
-    Quota.Counts counts = dir2Node.computeQuotaUsage();
-    assertEquals(3, counts.get(Quota.NAMESPACE));
-    assertEquals(0, counts.get(Quota.DISKSPACE));
+    QuotaCounts counts = dir2Node.computeQuotaUsage(fsdir.getBlockStoragePolicySuite());
+    assertEquals(3, counts.getNameSpace());
+    assertEquals(0, counts.getDiskSpace());
     childrenList = ReadOnlyList.Util.asList(dir2Node.asDirectory()
         .getChildrenList(Snapshot.CURRENT_STATE_ID));
     assertEquals(1, childrenList.size());
@@ -1790,9 +1790,10 @@ public class TestRenameWithSnapshots {
     // check dir2
     INode dir2Node = fsdir.getINode4Write(dir2.toString());
     assertTrue(dir2Node.asDirectory().isSnapshottable());
-    Quota.Counts counts = dir2Node.computeQuotaUsage();
-    assertEquals(4, counts.get(Quota.NAMESPACE));
-    assertEquals(BLOCKSIZE * REPL * 2, counts.get(Quota.DISKSPACE));
+    QuotaCounts counts = dir2Node.computeQuotaUsage(
+        fsdir.getBlockStoragePolicySuite());
+    assertEquals(4, counts.getNameSpace());
+    assertEquals(BLOCKSIZE * REPL * 2, counts.getDiskSpace());
   }
   
   @Test
@@ -1958,12 +1959,12 @@ public class TestRenameWithSnapshots {
     // check
     final INodeDirectory dir1Node = fsdir.getINode4Write(sdir1.toString())
         .asDirectory();
-    Quota.Counts q1 = dir1Node.getDirectoryWithQuotaFeature().getSpaceConsumed();  
-    assertEquals(3, q1.get(Quota.NAMESPACE));
+    QuotaCounts q1 = dir1Node.getDirectoryWithQuotaFeature().getSpaceConsumed();
+    assertEquals(3, q1.getNameSpace());
     final INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
         .asDirectory();
-    Quota.Counts q2 = dir2Node.getDirectoryWithQuotaFeature().getSpaceConsumed();  
-    assertEquals(1, q2.get(Quota.NAMESPACE));
+    QuotaCounts q2 = dir2Node.getDirectoryWithQuotaFeature().getSpaceConsumed();
+    assertEquals(1, q2.getNameSpace());
     
     final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
         foo.getName());
@@ -2028,12 +2029,12 @@ public class TestRenameWithSnapshots {
     final INodeDirectory dir1Node = fsdir.getINode4Write(sdir1.toString())
         .asDirectory();
     // sdir1 + s1 + foo_s1 (foo) + foo (foo + s1 + bar~bar3)
-    Quota.Counts q1 = dir1Node.getDirectoryWithQuotaFeature().getSpaceConsumed();  
-    assertEquals(7, q1.get(Quota.NAMESPACE));
+    QuotaCounts q1 = dir1Node.getDirectoryWithQuotaFeature().getSpaceConsumed();
+    assertEquals(7, q1.getNameSpace());
     final INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
         .asDirectory();
-    Quota.Counts q2 = dir2Node.getDirectoryWithQuotaFeature().getSpaceConsumed();  
-    assertEquals(1, q2.get(Quota.NAMESPACE));
+    QuotaCounts q2 = dir2Node.getDirectoryWithQuotaFeature().getSpaceConsumed();
+    assertEquals(1, q2.getNameSpace());
     
     final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
         foo.getName());

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

@@ -51,7 +51,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
-import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -203,17 +203,17 @@ public class TestSnapshotDeletion {
       final long expectedNs, final long expectedDs) throws IOException {
     INodeDirectory dirNode = getDir(fsdir, dirPath);
     assertTrue(dirNode.isQuotaSet());
-    Quota.Counts q = dirNode.getDirectoryWithQuotaFeature().getSpaceConsumed();
+    QuotaCounts q = dirNode.getDirectoryWithQuotaFeature().getSpaceConsumed();
     assertEquals(dirNode.dumpTreeRecursively().toString(), expectedNs,
-        q.get(Quota.NAMESPACE));
+        q.getNameSpace());
     assertEquals(dirNode.dumpTreeRecursively().toString(), expectedDs,
-        q.get(Quota.DISKSPACE));
-    Quota.Counts counts = Quota.Counts.newInstance();
-    dirNode.computeQuotaUsage(counts, false);
+        q.getDiskSpace());
+    QuotaCounts counts = new QuotaCounts.Builder().build();
+    dirNode.computeQuotaUsage(fsdir.getBlockStoragePolicySuite(), counts, false);
     assertEquals(dirNode.dumpTreeRecursively().toString(), expectedNs,
-        counts.get(Quota.NAMESPACE));
+        counts.getNameSpace());
     assertEquals(dirNode.dumpTreeRecursively().toString(), expectedDs,
-        counts.get(Quota.DISKSPACE));
+        counts.getDiskSpace());
   }
   
   /**

BIN
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored


+ 191 - 182
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

@@ -1,6 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <EDITS>
-  <EDITS_VERSION>-62</EDITS_VERSION>
+  <EDITS_VERSION>-63</EDITS_VERSION>
   <RECORD>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>
@@ -13,8 +13,8 @@
       <TXID>2</TXID>
       <DELEGATION_KEY>
         <KEY_ID>1</KEY_ID>
-        <EXPIRY_DATE>1422569009939</EXPIRY_DATE>
-        <KEY>907cb34000041937</KEY>
+        <EXPIRY_DATE>1423097579620</EXPIRY_DATE>
+        <KEY>ef3f2032e2797e8e</KEY>
       </DELEGATION_KEY>
     </DATA>
   </RECORD>
@@ -24,8 +24,8 @@
       <TXID>3</TXID>
       <DELEGATION_KEY>
         <KEY_ID>2</KEY_ID>
-        <EXPIRY_DATE>1422569009941</EXPIRY_DATE>
-        <KEY>178fa1bd83474b43</KEY>
+        <EXPIRY_DATE>1423097579622</EXPIRY_DATE>
+        <KEY>b978ed731a0b4a65</KEY>
       </DELEGATION_KEY>
     </DATA>
   </RECORD>
@@ -37,18 +37,18 @@
       <INODEID>16386</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877810832</MTIME>
-      <ATIME>1421877810832</ATIME>
+      <MTIME>1422406380345</MTIME>
+      <ATIME>1422406380345</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-986598042_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-156773767_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
       <RPC_CALLID>6</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -60,14 +60,14 @@
       <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877810888</MTIME>
-      <ATIME>1421877810832</ATIME>
+      <MTIME>1422406380369</MTIME>
+      <ATIME>1422406380345</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -78,10 +78,10 @@
     <DATA>
       <TXID>6</TXID>
       <PATH>/file_create</PATH>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-986598042_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-156773767_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <NEWBLOCK>false</NEWBLOCK>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
       <RPC_CALLID>8</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -93,14 +93,14 @@
       <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877810899</MTIME>
-      <ATIME>1421877810832</ATIME>
+      <MTIME>1422406380376</MTIME>
+      <ATIME>1422406380345</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -121,8 +121,8 @@
       <LENGTH>0</LENGTH>
       <SRC>/file_create</SRC>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1421877810907</TIMESTAMP>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
+      <TIMESTAMP>1422406380383</TIMESTAMP>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
       <RPC_CALLID>11</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -132,8 +132,8 @@
       <TXID>10</TXID>
       <LENGTH>0</LENGTH>
       <PATH>/file_moved</PATH>
-      <TIMESTAMP>1421877810915</TIMESTAMP>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
+      <TIMESTAMP>1422406380392</TIMESTAMP>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
       <RPC_CALLID>12</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -144,9 +144,9 @@
       <LENGTH>0</LENGTH>
       <INODEID>16387</INODEID>
       <PATH>/directory_mkdir</PATH>
-      <TIMESTAMP>1421877810923</TIMESTAMP>
+      <TIMESTAMP>1422406380399</TIMESTAMP>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>493</MODE>
       </PERMISSION_STATUS>
@@ -179,7 +179,7 @@
       <TXID>15</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
       <RPC_CALLID>17</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -190,7 +190,7 @@
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
       <RPC_CALLID>18</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -200,7 +200,7 @@
       <TXID>17</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
       <RPC_CALLID>19</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -212,18 +212,18 @@
       <INODEID>16388</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877810946</MTIME>
-      <ATIME>1421877810946</ATIME>
+      <MTIME>1422406380423</MTIME>
+      <ATIME>1422406380423</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-986598042_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-156773767_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
       <RPC_CALLID>20</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -235,14 +235,14 @@
       <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877810948</MTIME>
-      <ATIME>1421877810946</ATIME>
+      <MTIME>1422406380425</MTIME>
+      <ATIME>1422406380423</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -292,59 +292,68 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_RENAME</OPCODE>
+    <OPCODE>OP_SET_QUOTA_BY_STORAGETYPE</OPCODE>
     <DATA>
       <TXID>25</TXID>
+      <SRC>/directory_mkdir</SRC>
+      <STORAGETYPE>1</STORAGETYPE>
+      <DSQUOTA>888</DSQUOTA>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_RENAME</OPCODE>
+    <DATA>
+      <TXID>26</TXID>
       <LENGTH>0</LENGTH>
       <SRC>/file_create</SRC>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1421877810968</TIMESTAMP>
+      <TIMESTAMP>1422406380442</TIMESTAMP>
       <OPTIONS>NONE</OPTIONS>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>27</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>28</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TXID>26</TXID>
+      <TXID>27</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16389</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877810972</MTIME>
-      <ATIME>1421877810972</ATIME>
+      <MTIME>1422406380446</MTIME>
+      <ATIME>1422406380446</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-986598042_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-156773767_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>29</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>30</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>27</TXID>
+      <TXID>28</TXID>
       <BLOCK_ID>1073741825</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>28</TXID>
+      <TXID>29</TXID>
       <GENSTAMPV2>1001</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>29</TXID>
+      <TXID>30</TXID>
       <PATH>/file_concat_target</PATH>
       <BLOCK>
         <BLOCK_ID>1073741825</BLOCK_ID>
@@ -358,21 +367,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>30</TXID>
+      <TXID>31</TXID>
       <BLOCK_ID>1073741826</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>31</TXID>
+      <TXID>32</TXID>
       <GENSTAMPV2>1002</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>32</TXID>
+      <TXID>33</TXID>
       <PATH>/file_concat_target</PATH>
       <BLOCK>
         <BLOCK_ID>1073741825</BLOCK_ID>
@@ -391,21 +400,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>33</TXID>
+      <TXID>34</TXID>
       <BLOCK_ID>1073741827</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>34</TXID>
+      <TXID>35</TXID>
       <GENSTAMPV2>1003</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>35</TXID>
+      <TXID>36</TXID>
       <PATH>/file_concat_target</PATH>
       <BLOCK>
         <BLOCK_ID>1073741826</BLOCK_ID>
@@ -424,13 +433,13 @@
   <RECORD>
     <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TXID>36</TXID>
+      <TXID>37</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877811083</MTIME>
-      <ATIME>1421877810972</ATIME>
+      <MTIME>1422406380534</MTIME>
+      <ATIME>1422406380446</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -451,7 +460,7 @@
         <GENSTAMP>1003</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -460,44 +469,44 @@
   <RECORD>
     <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TXID>37</TXID>
+      <TXID>38</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16390</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877811086</MTIME>
-      <ATIME>1421877811086</ATIME>
+      <MTIME>1422406380537</MTIME>
+      <ATIME>1422406380537</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-986598042_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-156773767_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>39</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>40</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>38</TXID>
+      <TXID>39</TXID>
       <BLOCK_ID>1073741828</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>39</TXID>
+      <TXID>40</TXID>
       <GENSTAMPV2>1004</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>40</TXID>
+      <TXID>41</TXID>
       <PATH>/file_concat_0</PATH>
       <BLOCK>
         <BLOCK_ID>1073741828</BLOCK_ID>
@@ -511,21 +520,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>41</TXID>
+      <TXID>42</TXID>
       <BLOCK_ID>1073741829</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>42</TXID>
+      <TXID>43</TXID>
       <GENSTAMPV2>1005</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>43</TXID>
+      <TXID>44</TXID>
       <PATH>/file_concat_0</PATH>
       <BLOCK>
         <BLOCK_ID>1073741828</BLOCK_ID>
@@ -544,21 +553,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>44</TXID>
+      <TXID>45</TXID>
       <BLOCK_ID>1073741830</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>45</TXID>
+      <TXID>46</TXID>
       <GENSTAMPV2>1006</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>46</TXID>
+      <TXID>47</TXID>
       <PATH>/file_concat_0</PATH>
       <BLOCK>
         <BLOCK_ID>1073741829</BLOCK_ID>
@@ -577,13 +586,13 @@
   <RECORD>
     <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TXID>47</TXID>
+      <TXID>48</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877811108</MTIME>
-      <ATIME>1421877811086</ATIME>
+      <MTIME>1422406380558</MTIME>
+      <ATIME>1422406380537</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -604,7 +613,7 @@
         <GENSTAMP>1006</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -613,44 +622,44 @@
   <RECORD>
     <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TXID>48</TXID>
+      <TXID>49</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16391</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877811110</MTIME>
-      <ATIME>1421877811110</ATIME>
+      <MTIME>1422406380560</MTIME>
+      <ATIME>1422406380560</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-986598042_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-156773767_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>48</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>49</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>49</TXID>
+      <TXID>50</TXID>
       <BLOCK_ID>1073741831</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>50</TXID>
+      <TXID>51</TXID>
       <GENSTAMPV2>1007</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>51</TXID>
+      <TXID>52</TXID>
       <PATH>/file_concat_1</PATH>
       <BLOCK>
         <BLOCK_ID>1073741831</BLOCK_ID>
@@ -664,21 +673,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>52</TXID>
+      <TXID>53</TXID>
       <BLOCK_ID>1073741832</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>53</TXID>
+      <TXID>54</TXID>
       <GENSTAMPV2>1008</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>54</TXID>
+      <TXID>55</TXID>
       <PATH>/file_concat_1</PATH>
       <BLOCK>
         <BLOCK_ID>1073741831</BLOCK_ID>
@@ -697,21 +706,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>55</TXID>
+      <TXID>56</TXID>
       <BLOCK_ID>1073741833</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>56</TXID>
+      <TXID>57</TXID>
       <GENSTAMPV2>1009</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>57</TXID>
+      <TXID>58</TXID>
       <PATH>/file_concat_1</PATH>
       <BLOCK>
         <BLOCK_ID>1073741832</BLOCK_ID>
@@ -730,13 +739,13 @@
   <RECORD>
     <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TXID>58</TXID>
+      <TXID>59</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877811131</MTIME>
-      <ATIME>1421877811110</ATIME>
+      <MTIME>1422406380579</MTIME>
+      <ATIME>1422406380560</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -757,7 +766,7 @@
         <GENSTAMP>1009</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -766,59 +775,59 @@
   <RECORD>
     <OPCODE>OP_CONCAT_DELETE</OPCODE>
     <DATA>
-      <TXID>59</TXID>
+      <TXID>60</TXID>
       <LENGTH>0</LENGTH>
       <TRG>/file_concat_target</TRG>
-      <TIMESTAMP>1421877811134</TIMESTAMP>
+      <TIMESTAMP>1422406380582</TIMESTAMP>
       <SOURCES>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE2>/file_concat_1</SOURCE2>
       </SOURCES>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>56</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>57</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TXID>60</TXID>
+      <TXID>61</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16392</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877811137</MTIME>
-      <ATIME>1421877811137</ATIME>
+      <MTIME>1422406380586</MTIME>
+      <ATIME>1422406380586</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-986598042_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-156773767_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>58</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>59</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>61</TXID>
+      <TXID>62</TXID>
       <BLOCK_ID>1073741834</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>62</TXID>
+      <TXID>63</TXID>
       <GENSTAMPV2>1010</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>63</TXID>
+      <TXID>64</TXID>
       <PATH>/file_create</PATH>
       <BLOCK>
         <BLOCK_ID>1073741834</BLOCK_ID>
@@ -832,21 +841,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>64</TXID>
+      <TXID>65</TXID>
       <BLOCK_ID>1073741835</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>65</TXID>
+      <TXID>66</TXID>
       <GENSTAMPV2>1011</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>66</TXID>
+      <TXID>67</TXID>
       <PATH>/file_create</PATH>
       <BLOCK>
         <BLOCK_ID>1073741834</BLOCK_ID>
@@ -865,13 +874,13 @@
   <RECORD>
     <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TXID>67</TXID>
+      <TXID>68</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877811152</MTIME>
-      <ATIME>1421877811137</ATIME>
+      <MTIME>1422406380599</MTIME>
+      <ATIME>1422406380586</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -887,7 +896,7 @@
         <GENSTAMP>1011</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -896,74 +905,74 @@
   <RECORD>
     <OPCODE>OP_TRUNCATE</OPCODE>
     <DATA>
-      <TXID>68</TXID>
+      <TXID>69</TXID>
       <SRC>/file_create</SRC>
-      <CLIENTNAME>DFSClient_NONMAPREDUCE_-986598042_1</CLIENTNAME>
+      <CLIENTNAME>DFSClient_NONMAPREDUCE_-156773767_1</CLIENTNAME>
       <CLIENTMACHINE>127.0.0.1</CLIENTMACHINE>
       <NEWLENGTH>512</NEWLENGTH>
-      <TIMESTAMP>1421877811154</TIMESTAMP>
+      <TIMESTAMP>1422406380601</TIMESTAMP>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SYMLINK</OPCODE>
     <DATA>
-      <TXID>69</TXID>
+      <TXID>70</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16393</INODEID>
       <PATH>/file_symlink</PATH>
       <VALUE>/file_concat_target</VALUE>
-      <MTIME>1421877811160</MTIME>
-      <ATIME>1421877811160</ATIME>
+      <MTIME>1422406380606</MTIME>
+      <ATIME>1422406380606</ATIME>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>511</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>65</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>66</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TXID>70</TXID>
+      <TXID>71</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16394</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877811163</MTIME>
-      <ATIME>1421877811163</ATIME>
+      <MTIME>1422406380608</MTIME>
+      <ATIME>1422406380608</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-986598042_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-156773767_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>66</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>67</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>71</TXID>
+      <TXID>72</TXID>
       <BLOCK_ID>1073741836</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>72</TXID>
+      <TXID>73</TXID>
       <GENSTAMPV2>1012</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>73</TXID>
+      <TXID>74</TXID>
       <PATH>/hard-lease-recovery-test</PATH>
       <BLOCK>
         <BLOCK_ID>1073741836</BLOCK_ID>
@@ -977,7 +986,7 @@
   <RECORD>
     <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TXID>74</TXID>
+      <TXID>75</TXID>
       <PATH>/hard-lease-recovery-test</PATH>
       <BLOCK>
         <BLOCK_ID>1073741836</BLOCK_ID>
@@ -991,15 +1000,15 @@
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>75</TXID>
+      <TXID>76</TXID>
       <GENSTAMPV2>1013</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <DATA>
-      <TXID>76</TXID>
-      <LEASEHOLDER>DFSClient_NONMAPREDUCE_-986598042_1</LEASEHOLDER>
+      <TXID>77</TXID>
+      <LEASEHOLDER>DFSClient_NONMAPREDUCE_-156773767_1</LEASEHOLDER>
       <PATH>/hard-lease-recovery-test</PATH>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
     </DATA>
@@ -1007,13 +1016,13 @@
   <RECORD>
     <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TXID>77</TXID>
+      <TXID>78</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1421877813736</MTIME>
-      <ATIME>1421877811163</ATIME>
+      <MTIME>1422406383261</MTIME>
+      <ATIME>1422406380608</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -1024,7 +1033,7 @@
         <GENSTAMP>1013</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
-        <USERNAME>jing</USERNAME>
+        <USERNAME>xyao</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -1033,72 +1042,72 @@
   <RECORD>
     <OPCODE>OP_ADD_CACHE_POOL</OPCODE>
     <DATA>
-      <TXID>78</TXID>
+      <TXID>79</TXID>
       <POOLNAME>pool1</POOLNAME>
-      <OWNERNAME>jing</OWNERNAME>
+      <OWNERNAME>xyao</OWNERNAME>
       <GROUPNAME>staff</GROUPNAME>
       <MODE>493</MODE>
       <LIMIT>9223372036854775807</LIMIT>
       <MAXRELATIVEEXPIRY>2305843009213693951</MAXRELATIVEEXPIRY>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>73</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>74</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_MODIFY_CACHE_POOL</OPCODE>
     <DATA>
-      <TXID>79</TXID>
+      <TXID>80</TXID>
       <POOLNAME>pool1</POOLNAME>
       <LIMIT>99</LIMIT>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>74</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>75</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_CACHE_DIRECTIVE</OPCODE>
     <DATA>
-      <TXID>80</TXID>
+      <TXID>81</TXID>
       <ID>1</ID>
       <PATH>/path</PATH>
       <REPLICATION>1</REPLICATION>
       <POOL>pool1</POOL>
-      <EXPIRATION>2305844431091508160</EXPIRATION>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>75</RPC_CALLID>
+      <EXPIRATION>2305844431620077592</EXPIRATION>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>76</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_MODIFY_CACHE_DIRECTIVE</OPCODE>
     <DATA>
-      <TXID>81</TXID>
+      <TXID>82</TXID>
       <ID>1</ID>
       <REPLICATION>2</REPLICATION>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>76</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>77</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_REMOVE_CACHE_DIRECTIVE</OPCODE>
     <DATA>
-      <TXID>82</TXID>
+      <TXID>83</TXID>
       <ID>1</ID>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>77</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>78</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_REMOVE_CACHE_POOL</OPCODE>
     <DATA>
-      <TXID>83</TXID>
+      <TXID>84</TXID>
       <POOLNAME>pool1</POOLNAME>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>78</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>79</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_ACL</OPCODE>
     <DATA>
-      <TXID>84</TXID>
+      <TXID>85</TXID>
       <SRC>/file_concat_target</SRC>
       <ENTRY>
         <SCOPE>ACCESS</SCOPE>
@@ -1131,62 +1140,62 @@
   <RECORD>
     <OPCODE>OP_SET_XATTR</OPCODE>
     <DATA>
-      <TXID>85</TXID>
+      <TXID>86</TXID>
       <SRC>/file_concat_target</SRC>
       <XATTR>
         <NAMESPACE>USER</NAMESPACE>
         <NAME>a1</NAME>
         <VALUE>0x313233</VALUE>
       </XATTR>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>80</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>81</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_XATTR</OPCODE>
     <DATA>
-      <TXID>86</TXID>
+      <TXID>87</TXID>
       <SRC>/file_concat_target</SRC>
       <XATTR>
         <NAMESPACE>USER</NAMESPACE>
         <NAME>a2</NAME>
         <VALUE>0x373839</VALUE>
       </XATTR>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>81</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>82</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_REMOVE_XATTR</OPCODE>
     <DATA>
-      <TXID>87</TXID>
+      <TXID>88</TXID>
       <SRC>/file_concat_target</SRC>
       <XATTR>
         <NAMESPACE>USER</NAMESPACE>
         <NAME>a2</NAME>
       </XATTR>
-      <RPC_CLIENTID>1730855b-1f27-4f17-9f72-b9f92eb3a8bd</RPC_CLIENTID>
-      <RPC_CALLID>82</RPC_CALLID>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>83</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ROLLING_UPGRADE_START</OPCODE>
     <DATA>
-      <TXID>88</TXID>
-      <STARTTIME>1421877814254</STARTTIME>
+      <TXID>89</TXID>
+      <STARTTIME>1422406383706</STARTTIME>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ROLLING_UPGRADE_FINALIZE</OPCODE>
     <DATA>
-      <TXID>89</TXID>
-      <FINALIZETIME>1421877814254</FINALIZETIME>
+      <TXID>90</TXID>
+      <FINALIZETIME>1422406383706</FINALIZETIME>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_END_LOG_SEGMENT</OPCODE>
     <DATA>
-      <TXID>90</TXID>
+      <TXID>91</TXID>
     </DATA>
   </RECORD>
 </EDITS>