diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 9036ae23b7c2e..c2b8574286342 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -571,6 +571,9 @@ Release 2.7.0 - UNRELEASED HDFS-8035. Move checkBlocksProperlyReplicated() in FSNamesystem to BlockManager. (wheat9) + HDFS-7811. Avoid recursive call getStoragePolicyID in + INodeFile#computeQuotaUsage. (Xiaoyu Yao and jing9) + OPTIMIZATIONS HDFS-7454. Reduce memory footprint for AclEntries in NameNode. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java index e1ac17698f938..de7743aaf49df 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java @@ -860,23 +860,27 @@ private long loadEdits(Iterable editStreams, */ static void updateCountForQuota(BlockStoragePolicySuite bsps, INodeDirectory root) { - updateCountForQuotaRecursively(bsps, root, new QuotaCounts.Builder().build()); + updateCountForQuotaRecursively(bsps, root.getStoragePolicyID(), root, + new QuotaCounts.Builder().build()); } private static void updateCountForQuotaRecursively(BlockStoragePolicySuite bsps, - INodeDirectory dir, QuotaCounts counts) { + byte blockStoragePolicyId, INodeDirectory dir, QuotaCounts counts) { final long parentNamespace = counts.getNameSpace(); final long parentStoragespace = counts.getStorageSpace(); final EnumCounters parentTypeSpaces = counts.getTypeSpaces(); - dir.computeQuotaUsage4CurrentDirectory(bsps, counts); + dir.computeQuotaUsage4CurrentDirectory(bsps, blockStoragePolicyId, counts); for (INode child : dir.getChildrenList(Snapshot.CURRENT_STATE_ID)) { + final byte childPolicyId = child.getStoragePolicyIDForQuota(blockStoragePolicyId); if (child.isDirectory()) { - updateCountForQuotaRecursively(bsps, child.asDirectory(), counts); + updateCountForQuotaRecursively(bsps, childPolicyId, + child.asDirectory(), counts); } else { // file or symlink: count here to reduce recursive calls. - child.computeQuotaUsage(bsps, counts, false); + child.computeQuotaUsage(bsps, childPolicyId, counts, false, + Snapshot.CURRENT_STATE_ID); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java index 586cce4b1aa9c..e6294418ee9df 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java @@ -505,9 +505,14 @@ public final boolean isQuotaSet() { /** * Count subtree {@link Quota#NAMESPACE} and {@link Quota#STORAGESPACE} usages. + * Entry point for FSDirectory where blockStoragePolicyId is given its initial + * value. */ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) { - return computeQuotaUsage(bsps, new QuotaCounts.Builder().build(), true); + final byte storagePolicyId = isSymlink() ? + BlockStoragePolicySuite.ID_UNSPECIFIED : getStoragePolicyID(); + return computeQuotaUsage(bsps, storagePolicyId, + new QuotaCounts.Builder().build(), true, Snapshot.CURRENT_STATE_ID); } /** @@ -532,6 +537,7 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) { *
    *
    * @param bsps Block storage policy suite to calculate intended storage type usage
+   * @param blockStoragePolicyId block storage policy id of the current INode
    * @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.
@@ -542,12 +548,15 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
    * @return The same objects as the counts parameter.
    */
   public abstract QuotaCounts computeQuotaUsage(
-    BlockStoragePolicySuite bsps,
+    BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
     QuotaCounts counts, boolean useCache, int lastSnapshotId);
 
   public final QuotaCounts computeQuotaUsage(
     BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache) {
-    return computeQuotaUsage(bsps, counts, useCache, Snapshot.CURRENT_STATE_ID);
+    final byte storagePolicyId = isSymlink() ?
+        BlockStoragePolicySuite.ID_UNSPECIFIED : getStoragePolicyID();
+    return computeQuotaUsage(bsps, storagePolicyId, counts,
+        useCache, Snapshot.CURRENT_STATE_ID);
   }
 
   /**
@@ -707,6 +716,20 @@ public final INode setAccessTime(long accessTime, int latestSnapshotId) {
    */
   public abstract byte getLocalStoragePolicyID();
 
+  /**
+   * Get the storage policy ID while computing quota usage
+   * @param parentStoragePolicyId the storage policy ID of the parent directory
+   * @return the storage policy ID of this INode. Note that for an
+   * {@link INodeSymlink} we return {@link BlockStoragePolicySuite#ID_UNSPECIFIED}
+   * instead of throwing Exception
+   */
+  public byte getStoragePolicyIDForQuota(byte parentStoragePolicyId) {
+    byte localId = isSymlink() ?
+        BlockStoragePolicySuite.ID_UNSPECIFIED : getLocalStoragePolicyID();
+    return localId != BlockStoragePolicySuite.ID_UNSPECIFIED ?
+        localId : parentStoragePolicyId;
+  }
+
   /**
    * Breaks {@code path} into components.
    * @return array of byte arrays each of which represents
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
index dadb8c70652a0..9f55fc464b28b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
@@ -45,6 +45,8 @@
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
+import static org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite.ID_UNSPECIFIED;
+
 /**
  * Directory INode class.
  */
@@ -123,18 +125,18 @@ public byte getLocalStoragePolicyID() {
         return (xattr.getValue())[0];
       }
     }
-    return BlockStoragePolicySuite.ID_UNSPECIFIED;
+    return ID_UNSPECIFIED;
   }
 
   @Override
   public byte getStoragePolicyID() {
     byte id = getLocalStoragePolicyID();
-    if (id != BlockStoragePolicySuite.ID_UNSPECIFIED) {
+    if (id != ID_UNSPECIFIED) {
       return id;
     }
     // if it is unspecified, check its parent
     return getParent() != null ? getParent().getStoragePolicyID() :
-        BlockStoragePolicySuite.ID_UNSPECIFIED;
+        ID_UNSPECIFIED;
   }
 
   void setQuota(BlockStoragePolicySuite bsps, long nsQuota, long ssQuota, StorageType type) {
@@ -568,10 +570,11 @@ private void addChild(final INode node, final int insertionPoint) {
   }
 
   @Override
-  public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache,
+  public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
+      byte blockStoragePolicyId, QuotaCounts counts, boolean useCache,
       int lastSnapshotId) {
     final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
-    
+
     // we are computing the quota usage for a specific snapshot here, i.e., the
     // computation only includes files/directories that exist at the time of the
     // given snapshot
@@ -579,7 +582,9 @@ public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, QuotaCounts c
         && !(useCache && isQuotaSet())) {
       ReadOnlyList childrenList = getChildrenList(lastSnapshotId);
       for (INode child : childrenList) {
-        child.computeQuotaUsage(bsps, counts, useCache, lastSnapshotId);
+        final byte childPolicyId = child.getStoragePolicyIDForQuota(blockStoragePolicyId);
+        child.computeQuotaUsage(bsps, childPolicyId, counts, useCache,
+            lastSnapshotId);
       }
       counts.addNameSpace(1);
       return counts;
@@ -591,28 +596,33 @@ public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, QuotaCounts c
       return q.AddCurrentSpaceUsage(counts);
     } else {
       useCache = q != null && !q.isQuotaSet() ? false : useCache;
-      return computeDirectoryQuotaUsage(bsps, counts, useCache, lastSnapshotId);
+      return computeDirectoryQuotaUsage(bsps, blockStoragePolicyId, counts,
+          useCache, lastSnapshotId);
     }
   }
 
   private QuotaCounts computeDirectoryQuotaUsage(BlockStoragePolicySuite bsps,
-      QuotaCounts counts, boolean useCache, int lastSnapshotId) {
+      byte blockStoragePolicyId, QuotaCounts counts, boolean useCache,
+      int lastSnapshotId) {
     if (children != null) {
       for (INode child : children) {
-        child.computeQuotaUsage(bsps, counts, useCache, lastSnapshotId);
+        final byte childPolicyId = child.getStoragePolicyIDForQuota(blockStoragePolicyId);
+        child.computeQuotaUsage(bsps, childPolicyId, counts, useCache,
+            lastSnapshotId);
       }
     }
-    return computeQuotaUsage4CurrentDirectory(bsps, counts);
+    return computeQuotaUsage4CurrentDirectory(bsps, blockStoragePolicyId,
+        counts);
   }
   
   /** Add quota usage for this inode excluding children. */
   public QuotaCounts computeQuotaUsage4CurrentDirectory(
-      BlockStoragePolicySuite bsps, QuotaCounts counts) {
+      BlockStoragePolicySuite bsps, byte storagePolicyId, QuotaCounts counts) {
     counts.addNameSpace(1);
     // include the diff list
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     if (sf != null) {
-      sf.computeQuotaUsage4CurrentDirectory(bsps, counts);
+      sf.computeQuotaUsage4CurrentDirectory(bsps, storagePolicyId, counts);
     }
     return counts;
   }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 284633f9cd55a..08b7c73e3fb26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite.ID_UNSPECIFIED;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 
@@ -24,7 +25,6 @@
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -393,7 +393,7 @@ public byte getLocalStoragePolicyID() {
   @Override
   public byte getStoragePolicyID() {
     byte id = getLocalStoragePolicyID();
-    if (id == BlockStoragePolicySuite.ID_UNSPECIFIED) {
+    if (id == ID_UNSPECIFIED) {
       return this.getParent() != null ?
           this.getParent().getStoragePolicyID() : id;
     }
@@ -554,7 +554,8 @@ public String getName() {
   // derive the intended storage type usage for quota by storage type
   @Override
   public final QuotaCounts computeQuotaUsage(
-      BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache,
+      BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
+      QuotaCounts counts, boolean useCache,
       int lastSnapshotId) {
     long nsDelta = 1;
     final long ssDeltaNoReplication;
@@ -583,8 +584,8 @@ public final QuotaCounts computeQuotaUsage(
     counts.addNameSpace(nsDelta);
     counts.addStorageSpace(ssDeltaNoReplication * replication);
 
-    if (getStoragePolicyID() != BlockStoragePolicySuite.ID_UNSPECIFIED){
-      BlockStoragePolicy bsp = bsps.getPolicy(getStoragePolicyID());
+    if (blockStoragePolicyId != ID_UNSPECIFIED){
+      BlockStoragePolicy bsp = bsps.getPolicy(blockStoragePolicyId);
       List storageTypes = bsp.chooseStorageTypes(replication);
       for (StorageType t : storageTypes) {
         if (!t.supportTypeQuota()) {
@@ -618,7 +619,7 @@ public final ContentSummaryComputationContext computeContentSummary(
     counts.addContent(Content.LENGTH, fileLen);
     counts.addContent(Content.DISKSPACE, storagespaceConsumed());
 
-    if (getStoragePolicyID() != BlockStoragePolicySuite.ID_UNSPECIFIED){
+    if (getStoragePolicyID() != ID_UNSPECIFIED){
       BlockStoragePolicy bsp = summary.getBlockStoragePolicySuite().
           getPolicy(getStoragePolicyID());
       List storageTypes = bsp.chooseStorageTypes(getFileReplication());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
index cb270bfbc4c46..fc1cbec7fa7e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
@@ -103,8 +103,8 @@ public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
 
       @Override
       public QuotaCounts computeQuotaUsage(
-          BlockStoragePolicySuite bsps, QuotaCounts counts,
-          boolean useCache, int lastSnapshotId) {
+          BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
+          QuotaCounts counts, boolean useCache, int lastSnapshotId) {
         return null;
       }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
index eee50a52919b0..b33a93cb6a7ef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
@@ -23,7 +23,6 @@
 import java.util.Comparator;
 import java.util.List;
 
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@@ -327,9 +326,10 @@ public ContentSummaryComputationContext computeContentSummary(
 
   @Override
   public QuotaCounts computeQuotaUsage(
-    BlockStoragePolicySuite bsps,
+    BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
     QuotaCounts counts, boolean useCache, int lastSnapshotId) {
-    return referred.computeQuotaUsage(bsps, counts, useCache, lastSnapshotId);
+    return referred.computeQuotaUsage(bsps, blockStoragePolicyId, counts,
+        useCache, lastSnapshotId);
   }
 
   @Override
@@ -512,7 +512,8 @@ public final ContentSummaryComputationContext computeContentSummary(
         ContentSummaryComputationContext summary) {
       //only count storagespace for WithName
       final QuotaCounts q = new QuotaCounts.Builder().build();
-      computeQuotaUsage(summary.getBlockStoragePolicySuite(), q, false, lastSnapshotId);
+      computeQuotaUsage(summary.getBlockStoragePolicySuite(),
+          getStoragePolicyID(), q, false, lastSnapshotId);
       summary.getCounts().addContent(Content.DISKSPACE, q.getStorageSpace());
       summary.getCounts().addTypeSpaces(q.getTypeSpaces());
       return summary;
@@ -520,7 +521,8 @@ public final ContentSummaryComputationContext computeContentSummary(
 
     @Override
     public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
-        QuotaCounts counts, boolean useCache, int lastSnapshotId) {
+        byte blockStoragePolicyId, 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 
@@ -535,7 +537,8 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
       // been updated by changes in the current tree.
       int id = lastSnapshotId != Snapshot.CURRENT_STATE_ID ? 
           lastSnapshotId : this.lastSnapshotId;
-      return referred.computeQuotaUsage(bsps, counts, false, id);
+      return referred.computeQuotaUsage(bsps, blockStoragePolicyId, counts,
+          false, id);
     }
     
     @Override
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
index 120d0dcd76213..21a9e4f125a27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
@@ -93,7 +93,7 @@ public void destroyAndCollectBlocks(final BlockStoragePolicySuite bsps,
 
   @Override
   public QuotaCounts computeQuotaUsage(
-      BlockStoragePolicySuite bsps,
+      BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
       QuotaCounts counts, boolean useCache, int lastSnapshotId) {
     counts.addNameSpace(1);
     return counts;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
index d55332f544ad5..95f9d8abb69e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
@@ -641,10 +641,13 @@ public void clear(BlockStoragePolicySuite bsps, INodeDirectory currentINode,
   }
 
   public QuotaCounts computeQuotaUsage4CurrentDirectory(
-    BlockStoragePolicySuite bsps, QuotaCounts counts) {
+      BlockStoragePolicySuite bsps, byte storagePolicyId,
+      QuotaCounts counts) {
     for(DirectoryDiff d : diffs) {
       for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
-        deleted.computeQuotaUsage(bsps, counts, false, Snapshot.CURRENT_STATE_ID);
+        final byte childPolicyId = deleted.getStoragePolicyIDForQuota(storagePolicyId);
+        deleted.computeQuotaUsage(bsps, childPolicyId, counts, false,
+            Snapshot.CURRENT_STATE_ID);
       }
     }
     return counts;