Browse Source

HDFS-4208. NameNode could be stuck in SafeMode due to never-created blocks. Contributed by Brandon Li.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1@1413981 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 năm trước cách đây
mục cha
commit
2c16e89817

+ 3 - 0
CHANGES.txt

@@ -316,6 +316,9 @@ Release 1.2.0 - unreleased
     handles hostname starting with a numeric character.  (Jing Zhao via
     szetszwo)
 
+    HDFS-4208. NameNode could be stuck in SafeMode due to never-created
+    blocks. (Brandon Li via suresh)
+
 Release 1.1.1 - Unreleased
 
   INCOMPATIBLE CHANGES

+ 47 - 1
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -5339,7 +5339,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   void setBlockTotal() {
     if (safeMode == null)
       return;
-    safeMode.setBlockTotal(blocksMap.size());
+    safeMode.setBlockTotal((int)getSafeBlockCount());
   }
 
   /**
@@ -5349,6 +5349,52 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     return blocksMap.size();
   }
 
+  /**
+   * There are times when a block is allocated by a client but was never used to
+   * write to. This could happen because the response to block allocation
+   * request never made it to the client or the client failed right after block
+   * allocation. In such a case, NameNode might get stuck in safemode in
+   * subsequent restart waiting for such blocks to be reported. To handle this,
+   * such blocks should not be counted toward total blocks needed to exit
+   * safemode.
+   * <br>
+   * This method returns the total number of blocks excluding the last blocks of
+   * files under construction with length zero.
+   */
+  private long getSafeBlockCount() {
+    // Calculate number of blocks excluded in safe block count
+    long numExcludedBlocks = 0;
+    for (Lease lease : leaseManager.getSortedLeases()) {
+      for (String path : lease.getPaths()) {
+        INode node = dir.getFileINode(path);
+        if (node == null) {
+          LOG.error("Found a lease for nonexisting file: " + path);
+          continue;
+        }
+        if (!node.isUnderConstruction()) {
+          LOG.error("Found a lease for file that is not under construction:"
+              + path);
+          continue;
+        }
+        INodeFileUnderConstruction cons = (INodeFileUnderConstruction) node;
+        BlockInfo[] blocks = cons.getBlocks();
+        if (blocks == null || blocks.length == 0) {
+          continue;
+        }
+        // Exclude the last block of a file under construction with zero length
+        if (blocks[blocks.length - 1].getNumBytes() == 0) {
+          numExcludedBlocks++;
+        }
+      }
+    }
+    LOG.info("Number of blocks excluded by safe block count: "
+        + numExcludedBlocks + " total blocks: " + getBlocksTotal()
+        + " and thus the safe blocks: "
+        + (getBlocksTotal() - numExcludedBlocks));
+
+    return getBlocksTotal() - numExcludedBlocks;
+  }
+  
   /**
    * Enter safe mode manually.
    * @throws IOException

+ 48 - 0
src/test/org/apache/hadoop/hdfs/server/namenode/TestStartup.java

@@ -1,3 +1,21 @@
+/**
+ * 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 java.io.File;
@@ -15,12 +33,17 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeFile;
 import org.apache.hadoop.util.StringUtils;
+import org.junit.Test;
 
 /**
  * Startup and checkpoint tests
@@ -303,4 +326,29 @@ public class TestStartup extends TestCase {
         cluster.shutdown();
     }
   }
+  
+  /** Test SafeMode counts only complete blocks */
+  @Test(timeout=60000)
+  public void testGetBlocks() throws Exception {
+    final Configuration CONF = new Configuration();
+
+    config.set(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, "1.0f");
+    MiniDFSCluster cluster = new MiniDFSCluster(CONF, 2, true, null);
+    try {
+      cluster.waitActive();
+
+      // Create a file and add one block, but not write to DataNode
+      DFSClient client = new DFSClient(CONF);
+      client.namenode.create("/tmp1.txt", new FsPermission("755"),
+          "clientName", false, (short) 2, 1024);
+      client.namenode.addBlock("/tmp1.txt", "clientName", new DatanodeInfo[0]);
+
+      // Restart NameNode waiting for exiting safemode, ensure NameNode doesn't
+      // get stuck in safemode
+      cluster.restartNameNode();
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }