فهرست منبع

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

Ming Ma 9 سال پیش
والد
کامیت
3fa33b5c2c

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

@@ -2439,6 +2439,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9484. NNThroughputBenchmark$BlockReportStats should not send empty
     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
 
   INCOMPATIBLE CHANGES

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

@@ -563,25 +563,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     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
    */
@@ -1802,7 +1783,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   void concat(String target, String [] srcs, boolean logRetryCache)
       throws IOException {
-    waitForLoadingFSImage();
     HdfsFileStatus stat = null;
     boolean success = false;
     writeLock();
@@ -1899,7 +1879,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     if (!FileSystem.areSymlinksEnabled()) {
       throw new UnsupportedOperationException("Symlinks not supported");
     }
-    waitForLoadingFSImage();
     HdfsFileStatus auditStat = null;
     writeLock();
     try {
@@ -1933,7 +1912,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   boolean setReplication(final String src, final short replication)
       throws IOException {
     boolean success = false;
-    waitForLoadingFSImage();
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
@@ -1961,7 +1939,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   void setStoragePolicy(String src, String policyName) throws IOException {
     HdfsFileStatus auditStat;
-    waitForLoadingFSImage();
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
@@ -1988,7 +1965,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   BlockStoragePolicy getStoragePolicy(String src) throws IOException {
     checkOperation(OperationCategory.READ);
-    waitForLoadingFSImage();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
@@ -2003,7 +1979,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   BlockStoragePolicy[] getStoragePolicies() throws IOException {
     checkOperation(OperationCategory.READ);
-    waitForLoadingFSImage();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
@@ -2132,7 +2107,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     FSPermissionChecker pc = getPermissionChecker();
-    waitForLoadingFSImage();
 
     /**
      * If the file is in an encryption zone, we optimistically create an
@@ -2414,7 +2388,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     NameNode.stateChangeLog.debug("BLOCK* getAdditionalBlock: {}  inodeId {}" +
         " for {}", src, fileId, clientName);
 
-    waitForLoadingFSImage();
     LocatedBlock[] onRetryBlock = new LocatedBlock[1];
     FSDirWriteFileOp.ValidateAddBlockResult r;
     FSPermissionChecker pc = getPermissionChecker();
@@ -2525,7 +2498,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     NameNode.stateChangeLog.debug(
         "BLOCK* NameSystem.abandonBlock: {} of file {}", b, src);
-    waitForLoadingFSImage();
     checkOperation(OperationCategory.WRITE);
     FSPermissionChecker pc = getPermissionChecker();
     writeLock();
@@ -2593,7 +2565,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     throws IOException {
     boolean success = false;
     checkOperation(OperationCategory.WRITE);
-    waitForLoadingFSImage();
     FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
@@ -2651,7 +2622,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   @Deprecated
   boolean renameTo(String src, String dst, boolean logRetryCache)
       throws IOException {
-    waitForLoadingFSImage();
     FSDirRenameOp.RenameOldResult ret = null;
     writeLock();
     try {
@@ -2676,7 +2646,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void renameTo(final String src, final String dst,
                 boolean logRetryCache, Options.Rename... options)
       throws IOException {
-    waitForLoadingFSImage();
     Map.Entry<BlocksMapUpdateInfo, HdfsFileStatus> res = null;
     writeLock();
     try {
@@ -2712,7 +2681,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   boolean delete(String src, boolean recursive, boolean logRetryCache)
       throws IOException {
-    waitForLoadingFSImage();
     BlocksMapUpdateInfo toRemovedBlocks = null;
     writeLock();
     boolean ret = false;
@@ -2940,7 +2908,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
 
     FSPermissionChecker pc = getPermissionChecker();
-    waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -3193,7 +3160,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // since we just remove the uc feature from pendingFile
     pendingFile.toCompleteFile(now());
 
-    waitForLoadingFSImage();
     // close file and persist block allocations for this file
     closeFile(src, pendingFile);
 
@@ -3256,7 +3222,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
              + ")");
     checkOperation(OperationCategory.WRITE);
     final String src;
-    waitForLoadingFSImage();
     writeLock();
     boolean copyTruncate = false;
     BlockInfo truncatedBlock = null;
@@ -3602,7 +3567,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   private void closeFile(String path, INodeFile file) {
     assert hasWriteLock();
-    waitForLoadingFSImage();
     // file is closed
     getEditLog().logCloseFile(path, file);
     NameNode.stateChangeLog.debug("closeFile: {} with {} blocks is persisted" +
@@ -4748,7 +4712,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
              + ", newNodes=" + Arrays.asList(newNodes)
              + ", client=" + clientName
              + ")");
-    waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -6591,7 +6554,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
     checkOperation(OperationCategory.READ);
-    waitForLoadingFSImage();
     readLock();
     try {
       checkOperation(OperationCategory.READ);