You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ec...@apache.org on 2015/09/15 21:12:00 UTC

[15/50] [abbrv] hadoop git commit: HDFS-6763. Initialize file system-wide quota once on transitioning to active. Contributed by Kihwal Lee

HDFS-6763. Initialize file system-wide quota once on transitioning to active. Contributed by Kihwal Lee


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a40342b0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a40342b0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a40342b0

Branch: refs/heads/HADOOP-11890
Commit: a40342b0dab1f9137ae4b3679a5aca7f2a57d23d
Parents: 7b5b2c5
Author: Kihwal Lee <ki...@apache.org>
Authored: Thu Sep 10 09:16:29 2015 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Thu Sep 10 09:16:29 2015 -0500

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/BackupImage.java       |   8 +-
 .../hdfs/server/namenode/FSDirectory.java       | 128 +++++++++++++++++++
 .../hadoop/hdfs/server/namenode/FSImage.java    | 126 ------------------
 .../hdfs/server/namenode/FSNamesystem.java      |   2 +
 .../namenode/TestDiskspaceQuotaUpdate.java      |   9 +-
 .../namenode/TestFSImageWithSnapshot.java       |   3 +-
 7 files changed, 138 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a40342b0/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 445c50f..e241460 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -914,6 +914,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8974. Convert docs in xdoc format to markdown.
     (Masatake Iwasaki via aajisaka)
 
+    HDFS-6763. Initialize file system-wide quota once on transitioning to active
+    (kihwal)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a40342b0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
index 8aee0bb..c6ae0d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
@@ -24,7 +24,6 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -94,9 +93,6 @@ public class BackupImage extends FSImage {
     super(conf);
     storage.setDisablePreUpgradableLayoutCheck(true);
     bnState = BNState.DROP_UNTIL_NEXT_ROLL;
-    quotaInitThreads = conf.getInt(
-        DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_KEY,
-        DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_DEFAULT);
   }
 
   synchronized FSNamesystem getNamesystem() {
@@ -222,9 +218,7 @@ public class BackupImage extends FSImage {
       }
       lastAppliedTxId = logLoader.getLastAppliedTxId();
 
-      FSImage.updateCountForQuota(
-          getNamesystem().dir.getBlockStoragePolicySuite(),
-          getNamesystem().dir.rootDir, quotaInitThreads);
+      getNamesystem().dir.updateCountForQuota();
     } finally {
       backupInputStream.clear();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a40342b0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 8c74e48..e25e0e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -57,8 +57,10 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo.UpdatedReplicationInfo;
 import org.apache.hadoop.hdfs.util.ByteArray;
 import org.apache.hadoop.hdfs.util.EnumCounters;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -68,6 +70,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.RecursiveAction;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
@@ -138,6 +142,7 @@ public class FSDirectory implements Closeable {
   private final long contentSleepMicroSec;
   private final INodeMap inodeMap; // Synchronized by dirLock
   private long yieldCount = 0; // keep track of lock yield count.
+  private int quotaInitThreads;
 
   private final int inodeXAttrsLimit; //inode xattrs max limit
 
@@ -312,6 +317,10 @@ public class FSDirectory implements Closeable {
     namesystem = ns;
     this.editLog = ns.getEditLog();
     ezManager = new EncryptionZoneManager(this, conf);
+
+    this.quotaInitThreads = conf.getInt(
+        DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_KEY,
+        DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_DEFAULT);
   }
     
   FSNamesystem getFSNamesystem() {
@@ -503,6 +512,125 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  /**
+   * Update the count of each directory with quota in the namespace.
+   * A directory's count is defined as the total number inodes in the tree
+   * rooted at the directory.
+   *
+   * This is an update of existing state of the filesystem and does not
+   * throw QuotaExceededException.
+   */
+  void updateCountForQuota(int initThreads) {
+    writeLock();
+    try {
+      int threads = (initThreads < 1) ? 1 : initThreads;
+      LOG.info("Initializing quota with " + threads + " thread(s)");
+      long start = Time.now();
+      QuotaCounts counts = new QuotaCounts.Builder().build();
+      ForkJoinPool p = new ForkJoinPool(threads);
+      RecursiveAction task = new InitQuotaTask(getBlockStoragePolicySuite(),
+          rootDir.getStoragePolicyID(), rootDir, counts);
+      p.execute(task);
+      task.join();
+      p.shutdown();
+      LOG.info("Quota initialization completed in " + (Time.now() - start) +
+          " milliseconds\n" + counts);
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  void updateCountForQuota() {
+    updateCountForQuota(quotaInitThreads);
+  }
+
+  /**
+   * parallel initialization using fork-join.
+   */
+  private static class InitQuotaTask extends RecursiveAction {
+    private final INodeDirectory dir;
+    private final QuotaCounts counts;
+    private final BlockStoragePolicySuite bsps;
+    private final byte blockStoragePolicyId;
+
+    public InitQuotaTask(BlockStoragePolicySuite bsps,
+        byte blockStoragePolicyId, INodeDirectory dir, QuotaCounts counts) {
+      this.dir = dir;
+      this.counts = counts;
+      this.bsps = bsps;
+      this.blockStoragePolicyId = blockStoragePolicyId;
+    }
+
+    public void compute() {
+      QuotaCounts myCounts =  new QuotaCounts.Builder().build();
+      dir.computeQuotaUsage4CurrentDirectory(bsps, blockStoragePolicyId,
+          myCounts);
+
+      ReadOnlyList<INode> children =
+          dir.getChildrenList(CURRENT_STATE_ID);
+
+      if (children.size() > 0) {
+        List<InitQuotaTask> subtasks = new ArrayList<InitQuotaTask>();
+        for (INode child : children) {
+          final byte childPolicyId =
+              child.getStoragePolicyIDForQuota(blockStoragePolicyId);
+          if (child.isDirectory()) {
+            subtasks.add(new InitQuotaTask(bsps, childPolicyId,
+                child.asDirectory(), myCounts));
+          } else {
+            // file or symlink. count using the local counts variable
+            myCounts.add(child.computeQuotaUsage(bsps, childPolicyId, false,
+                CURRENT_STATE_ID));
+          }
+        }
+        // invoke and wait for completion
+        invokeAll(subtasks);
+      }
+
+      if (dir.isQuotaSet()) {
+        // check if quota is violated. It indicates a software bug.
+        final QuotaCounts q = dir.getQuotaCounts();
+
+        final long nsConsumed = myCounts.getNameSpace();
+        final long nsQuota = q.getNameSpace();
+        if (Quota.isViolated(nsQuota, nsConsumed)) {
+          LOG.warn("Namespace quota violation in image for "
+              + dir.getFullPathName()
+              + " quota = " + nsQuota + " < consumed = " + nsConsumed);
+        }
+
+        final long ssConsumed = myCounts.getStorageSpace();
+        final long ssQuota = q.getStorageSpace();
+        if (Quota.isViolated(ssQuota, ssConsumed)) {
+          LOG.warn("Storagespace quota violation in image for "
+              + dir.getFullPathName()
+              + " quota = " + ssQuota + " < consumed = " + ssConsumed);
+        }
+
+        final EnumCounters<StorageType> tsConsumed = myCounts.getTypeSpaces();
+        for (StorageType t : StorageType.getTypesSupportingQuota()) {
+          final long typeSpace = tsConsumed.get(t);
+          final long typeQuota = q.getTypeSpaces().get(t);
+          if (Quota.isViolated(typeQuota, typeSpace)) {
+            LOG.warn("Storage type quota violation in image for "
+                + dir.getFullPathName()
+                + " type = " + t.toString() + " quota = "
+                + typeQuota + " < consumed " + typeSpace);
+          }
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Setting quota for " + dir + "\n" + myCounts);
+        }
+        dir.getDirectoryWithQuotaFeature().setSpaceConsumed(nsConsumed,
+            ssConsumed, tsConsumed);
+      }
+
+      synchronized(counts) {
+        counts.add(myCounts);
+      }
+    }
+  }
+
   /** Updates namespace, storagespace and typespaces consumed for all
    * directories until the parent directory of file represented by path.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a40342b0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
----------------------------------------------------------------------
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 0dd007d..93dc097 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
@@ -27,8 +27,6 @@ import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.concurrent.ForkJoinPool;
-import java.util.concurrent.RecursiveAction;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -42,12 +40,10 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
@@ -61,7 +57,6 @@ import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
@@ -70,9 +65,7 @@ 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.util.Canceler;
-import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
-import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.util.Time;
 
@@ -147,12 +140,7 @@ public class FSImage implements Closeable {
       storage.setRestoreFailedStorage(true);
     }
 
-    this.quotaInitThreads = conf.getInt(
-        DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_KEY,
-        DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_DEFAULT);
-
     this.editLog = new FSEditLog(conf, storage, editsDirs);
-    
     archivalManager = new NNStorageRetentionManager(conf, storage, editLog);
   }
  
@@ -853,126 +841,12 @@ public class FSImage implements Closeable {
       }
     } finally {
       FSEditLog.closeAllStreams(editStreams);
-      // update the counts
-      updateCountForQuota(target.getBlockManager().getStoragePolicySuite(),
-          target.dir.rootDir, quotaInitThreads);
     }
     prog.endPhase(Phase.LOADING_EDITS);
     return lastAppliedTxId - prevLastAppliedTxId;
   }
 
   /**
-   * Update the count of each directory with quota in the namespace.
-   * A directory's count is defined as the total number inodes in the tree
-   * rooted at the directory.
-   * 
-   * This is an update of existing state of the filesystem and does not
-   * throw QuotaExceededException.
-   */
-  static void updateCountForQuota(BlockStoragePolicySuite bsps,
-      INodeDirectory root, int threads) {
-    threads = (threads < 1) ? 1 : threads;
-    LOG.info("Initializing quota with " + threads + " thread(s)");
-    long start = Time.now();
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-    ForkJoinPool p = new ForkJoinPool(threads);
-    RecursiveAction task = new InitQuotaTask(bsps, root.getStoragePolicyID(),
-        root, counts);
-    p.execute(task);
-    task.join();
-    p.shutdown();
-    LOG.info("Quota initialization completed in " + (Time.now() - start) +
-        " milliseconds\n" + counts);
-  }
-
-  /**
-   * parallel initialization using fork-join.
-   */
-  private static class InitQuotaTask extends RecursiveAction {
-    private final INodeDirectory dir;
-    private final QuotaCounts counts;
-    private final BlockStoragePolicySuite bsps;
-    private final byte blockStoragePolicyId;
-
-    public InitQuotaTask(BlockStoragePolicySuite bsps,
-        byte blockStoragePolicyId, INodeDirectory dir, QuotaCounts counts) {
-      this.dir = dir;
-      this.counts = counts;
-      this.bsps = bsps;
-      this.blockStoragePolicyId = blockStoragePolicyId;
-    }
-
-    public void compute() {
-      QuotaCounts myCounts =  new QuotaCounts.Builder().build();
-      dir.computeQuotaUsage4CurrentDirectory(bsps, blockStoragePolicyId,
-          myCounts);
-
-      ReadOnlyList<INode> children =
-          dir.getChildrenList(Snapshot.CURRENT_STATE_ID);
-
-      if (children.size() > 0) {
-        List<InitQuotaTask> subtasks = new ArrayList<InitQuotaTask>();
-        for (INode child : children) {
-          final byte childPolicyId =
-              child.getStoragePolicyIDForQuota(blockStoragePolicyId);
-          if (child.isDirectory()) {
-            subtasks.add(new InitQuotaTask(bsps, childPolicyId,
-                child.asDirectory(), myCounts));
-          } else {
-            // file or symlink. count using the local counts variable
-            myCounts.add(child.computeQuotaUsage(bsps, childPolicyId, false,
-                Snapshot.CURRENT_STATE_ID));
-          }
-        }
-        // invoke and wait for completion
-        invokeAll(subtasks);
-      }
-
-      if (dir.isQuotaSet()) {
-        // check if quota is violated. It indicates a software bug.
-        final QuotaCounts q = dir.getQuotaCounts();
-
-        final long nsConsumed = myCounts.getNameSpace();
-        final long nsQuota = q.getNameSpace();
-        if (Quota.isViolated(nsQuota, nsConsumed)) {
-          LOG.warn("Namespace quota violation in image for "
-              + dir.getFullPathName()
-              + " quota = " + nsQuota + " < consumed = " + nsConsumed);
-        }
-
-        final long ssConsumed = myCounts.getStorageSpace();
-        final long ssQuota = q.getStorageSpace();
-        if (Quota.isViolated(ssQuota, ssConsumed)) {
-          LOG.warn("Storagespace quota violation in image for "
-              + dir.getFullPathName()
-              + " quota = " + ssQuota + " < consumed = " + ssConsumed);
-        }
-
-        final EnumCounters<StorageType> tsConsumed = myCounts.getTypeSpaces();
-        for (StorageType t : StorageType.getTypesSupportingQuota()) {
-          final long typeSpace = tsConsumed.get(t);
-          final long typeQuota = q.getTypeSpaces().get(t);
-          if (Quota.isViolated(typeQuota, typeSpace)) {
-            LOG.warn("Storage type quota violation in image for "
-                + dir.getFullPathName()
-                + " type = " + t.toString() + " quota = "
-                + typeQuota + " < consumed " + typeSpace);
-          }
-        }
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Setting quota for " + dir + "\n" + myCounts);
-        }
-        dir.getDirectoryWithQuotaFeature().setSpaceConsumed(nsConsumed,
-            ssConsumed, tsConsumed);
-      }
-
-      synchronized(counts) {
-        counts.add(myCounts);
-      }
-    }
-  }
-
-  /**
    * Load the image namespace from the given image file, verifying
    * it against the MD5 sum stored in its associated .md5 file.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a40342b0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 7d766b9..328c29d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1117,6 +1117,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         getFSImage().editLog.openForWrite(getEffectiveLayoutVersion());
       }
 
+      // Initialize the quota.
+      dir.updateCountForQuota();
       // Enable quota checks.
       dir.enableQuotaChecks();
       if (haEnabled) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a40342b0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java
index 0765a22..cf64638 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java
@@ -339,16 +339,13 @@ public class TestDiskspaceQuotaUpdate {
     HashMap<String, Long> dsMap = new HashMap<String, Long>();
     scanDirsWithQuota(root, nsMap, dsMap, false);
 
-    FSImage.updateCountForQuota(
-        fsdir.getBlockManager().getStoragePolicySuite(), root, 1);
+    fsdir.updateCountForQuota(1);
     scanDirsWithQuota(root, nsMap, dsMap, true);
 
-    FSImage.updateCountForQuota(
-        fsdir.getBlockManager().getStoragePolicySuite(), root, 2);
+    fsdir.updateCountForQuota(2);
     scanDirsWithQuota(root, nsMap, dsMap, true);
 
-    FSImage.updateCountForQuota(
-        fsdir.getBlockManager().getStoragePolicySuite(), root, 4);
+    fsdir.updateCountForQuota(4);
     scanDirsWithQuota(root, nsMap, dsMap, true);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a40342b0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
index 61b7f7c..1ff18a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
@@ -158,8 +158,7 @@ public class TestFSImageWithSnapshot {
     fsn.getFSDirectory().writeLock();
     try {
       loader.load(imageFile, false);
-      FSImage.updateCountForQuota(fsn.getBlockManager().getStoragePolicySuite(),
-          INodeDirectory.valueOf(fsn.getFSDirectory().getINode("/"), "/"), 4);
+      fsn.getFSDirectory().updateCountForQuota();
     } finally {
       fsn.getFSDirectory().writeUnlock();
       fsn.writeUnlock();