浏览代码

HDFS-9430 Remove waitForLoadingFSImage since checkNNStartup has ensured image loaded and namenode started. (Brahma Reddy Battula via mingma)

(cherry picked from commit 3fa33b5c2c289ceaced30c6c5451f3569110459d)
Ming Ma 9 年之前
父节点
当前提交
a68faf1a02

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

@@ -1574,6 +1574,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9484. NNThroughputBenchmark$BlockReportStats should not send empty
     HDFS-9484. NNThroughputBenchmark$BlockReportStats should not send empty
     block reports. (Mingliang Liu via shv)
     block reports. (Mingliang Liu via shv)
 
 
+    HDFS-9430. Remove waitForLoadingFSImage since checkNNStartup has ensured
+    image loaded and namenode started. (Brahma Reddy Battula via mingma)
+
 Release 2.7.3 - UNRELEASED
 Release 2.7.3 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 0 - 36
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -552,25 +552,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     imageLoaded = flag;
     imageLoaded = flag;
   }
   }
 
 
-  /**
-   * Block until the object is imageLoaded to be used.
-   */
-  void waitForLoadingFSImage() {
-    if (!imageLoaded) {
-      writeLock();
-      try {
-        while (!imageLoaded) {
-          try {
-            cond.await(5000, TimeUnit.MILLISECONDS);
-          } catch (InterruptedException ignored) {
-          }
-        }
-      } finally {
-        writeUnlock();
-      }
-    }
-  }
-
   /**
   /**
    * Clear all loaded data
    * Clear all loaded data
    */
    */
@@ -1800,7 +1781,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
    */
   void concat(String target, String [] srcs, boolean logRetryCache)
   void concat(String target, String [] srcs, boolean logRetryCache)
       throws IOException {
       throws IOException {
-    waitForLoadingFSImage();
     HdfsFileStatus stat = null;
     HdfsFileStatus stat = null;
     boolean success = false;
     boolean success = false;
     writeLock();
     writeLock();
@@ -1885,7 +1865,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   boolean setReplication(final String src, final short replication)
   boolean setReplication(final String src, final short replication)
       throws IOException {
       throws IOException {
     boolean success = false;
     boolean success = false;
-    waitForLoadingFSImage();
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
     writeLock();
     writeLock();
     try {
     try {
@@ -1960,7 +1939,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
    */
   void setStoragePolicy(String src, String policyName) throws IOException {
   void setStoragePolicy(String src, String policyName) throws IOException {
     HdfsFileStatus auditStat;
     HdfsFileStatus auditStat;
-    waitForLoadingFSImage();
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
     writeLock();
     writeLock();
     try {
     try {
@@ -1987,7 +1965,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
    */
   BlockStoragePolicy getStoragePolicy(String src) throws IOException {
   BlockStoragePolicy getStoragePolicy(String src) throws IOException {
     checkOperation(OperationCategory.READ);
     checkOperation(OperationCategory.READ);
-    waitForLoadingFSImage();
     readLock();
     readLock();
     try {
     try {
       checkOperation(OperationCategory.READ);
       checkOperation(OperationCategory.READ);
@@ -2002,7 +1979,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
    */
   BlockStoragePolicy[] getStoragePolicies() throws IOException {
   BlockStoragePolicy[] getStoragePolicies() throws IOException {
     checkOperation(OperationCategory.READ);
     checkOperation(OperationCategory.READ);
-    waitForLoadingFSImage();
     readLock();
     readLock();
     try {
     try {
       checkOperation(OperationCategory.READ);
       checkOperation(OperationCategory.READ);
@@ -2119,7 +2095,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
     }
 
 
     FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker pc = getPermissionChecker();
-    waitForLoadingFSImage();
 
 
     /**
     /**
      * If the file is in an encryption zone, we optimistically create an
      * If the file is in an encryption zone, we optimistically create an
@@ -2407,7 +2382,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     NameNode.stateChangeLog.debug("BLOCK* getAdditionalBlock: {} inodeId {}" +
     NameNode.stateChangeLog.debug("BLOCK* getAdditionalBlock: {} inodeId {}" +
         " for {}", src, fileId, clientName);
         " for {}", src, fileId, clientName);
 
 
-    waitForLoadingFSImage();
     LocatedBlock[] onRetryBlock = new LocatedBlock[1];
     LocatedBlock[] onRetryBlock = new LocatedBlock[1];
     FSDirWriteFileOp.ValidateAddBlockResult r;
     FSDirWriteFileOp.ValidateAddBlockResult r;
     FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker pc = getPermissionChecker();
@@ -2516,7 +2490,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
       throws IOException {
     NameNode.stateChangeLog.debug(
     NameNode.stateChangeLog.debug(
         "BLOCK* NameSystem.abandonBlock: {} of file {}", b, src);
         "BLOCK* NameSystem.abandonBlock: {} of file {}", b, src);
-    waitForLoadingFSImage();
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
     FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     writeLock();
@@ -2583,7 +2556,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     throws IOException {
     throws IOException {
     boolean success = false;
     boolean success = false;
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
-    waitForLoadingFSImage();
     FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     writeLock();
     try {
     try {
@@ -2636,7 +2608,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   @Deprecated
   @Deprecated
   boolean renameTo(String src, String dst, boolean logRetryCache)
   boolean renameTo(String src, String dst, boolean logRetryCache)
       throws IOException {
       throws IOException {
-    waitForLoadingFSImage();
     FSDirRenameOp.RenameOldResult ret = null;
     FSDirRenameOp.RenameOldResult ret = null;
     writeLock();
     writeLock();
     try {
     try {
@@ -2661,7 +2632,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void renameTo(final String src, final String dst,
   void renameTo(final String src, final String dst,
                 boolean logRetryCache, Options.Rename... options)
                 boolean logRetryCache, Options.Rename... options)
       throws IOException {
       throws IOException {
-    waitForLoadingFSImage();
     Map.Entry<BlocksMapUpdateInfo, HdfsFileStatus> res = null;
     Map.Entry<BlocksMapUpdateInfo, HdfsFileStatus> res = null;
     writeLock();
     writeLock();
     try {
     try {
@@ -2697,7 +2667,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
    */
   boolean delete(String src, boolean recursive, boolean logRetryCache)
   boolean delete(String src, boolean recursive, boolean logRetryCache)
       throws IOException {
       throws IOException {
-    waitForLoadingFSImage();
     BlocksMapUpdateInfo toRemovedBlocks = null;
     BlocksMapUpdateInfo toRemovedBlocks = null;
     writeLock();
     writeLock();
     boolean ret = false;
     boolean ret = false;
@@ -2973,7 +2942,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
 
 
     FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker pc = getPermissionChecker();
-    waitForLoadingFSImage();
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
@@ -3205,7 +3173,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // since we just remove the uc feature from pendingFile
     // since we just remove the uc feature from pendingFile
     pendingFile.toCompleteFile(now());
     pendingFile.toCompleteFile(now());
 
 
-    waitForLoadingFSImage();
     // close file and persist block allocations for this file
     // close file and persist block allocations for this file
     closeFile(src, pendingFile);
     closeFile(src, pendingFile);
 
 
@@ -3268,7 +3235,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
              + ")");
              + ")");
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
     final String src;
     final String src;
-    waitForLoadingFSImage();
     writeLock();
     writeLock();
     boolean copyTruncate = false;
     boolean copyTruncate = false;
     BlockInfo truncatedBlock = null;
     BlockInfo truncatedBlock = null;
@@ -3615,7 +3581,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
    */
   private void closeFile(String path, INodeFile file) {
   private void closeFile(String path, INodeFile file) {
     assert hasWriteLock();
     assert hasWriteLock();
-    waitForLoadingFSImage();
     // file is closed
     // file is closed
     getEditLog().logCloseFile(path, file);
     getEditLog().logCloseFile(path, file);
     NameNode.stateChangeLog.debug("closeFile: {} with {} bloks is persisted" +
     NameNode.stateChangeLog.debug("closeFile: {} with {} bloks is persisted" +
@@ -5382,7 +5347,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
              + ", newNodes=" + Arrays.asList(newNodes)
              + ", newNodes=" + Arrays.asList(newNodes)
              + ", client=" + clientName
              + ", client=" + clientName
              + ")");
              + ")");
-    waitForLoadingFSImage();
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);