Browse Source

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

Arpit Agarwal 10 năm trước cách đây
mục cha
commit
676fc2d1dc
52 tập tin đã thay đổi với 2160 bổ sung719 xóa
  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>