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 sh...@apache.org on 2008/12/20 01:33:14 UTC

svn commit: r728195 - in /hadoop/core/trunk: CHANGES.txt src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java src/test/org/apache/hadoop/hdfs/TestSafeMode.java

Author: shv
Date: Fri Dec 19 16:33:14 2008
New Revision: 728195

URL: http://svn.apache.org/viewvc?rev=728195&view=rev
Log:
HADOOP-1980. 'dfsadmin -safemode enter' should prevent the namenode from leaving safemode automatically. Contributed by Konstantin Shvachko & Raghu Angadi.

Added:
    hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestSafeMode.java
Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=728195&r1=728194&r2=728195&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Fri Dec 19 16:33:14 2008
@@ -1589,6 +1589,9 @@
 
     HADOOP-4904. Fix deadlock while leaving safe mode. (shv)
 
+    HADOOP-1980. 'dfsadmin -safemode enter' should prevent the namenode from
+    leaving safemode automatically. (shv & Raghu Angadi)
+
 Release 0.18.2 - 2008-11-03
 
   BUG FIXES

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=728195&r1=728194&r2=728195&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Fri Dec 19 16:33:14 2008
@@ -3835,7 +3835,7 @@
      */
     private SafeModeInfo() {
       this.threshold = 1.5f;  // this threshold can never be reached
-      this.extension = 0;
+      this.extension = Integer.MAX_VALUE;
       this.safeReplication = Short.MAX_VALUE + 1; // more than maxReplication
       this.blockTotal = -1;
       this.blockSafe = -1;
@@ -3993,33 +3993,42 @@
         this.blockSafe--;
       checkMode();
     }
-      
+
     /**
      * Check if safe mode was entered manually or at startup.
      */
     boolean isManual() {
-      return blockTotal == -1;
+      return extension == Integer.MAX_VALUE;
     }
-      
+
+    /**
+     * Set manual safe mode.
+     */
+    void setManual() {
+      extension = Integer.MAX_VALUE;
+    }
+
     /**
      * A tip on how safe mode is to be turned off: manually or automatically.
      */
     String getTurnOffTip() {
-      final String autoOffMsg = "Safe mode will be turned off automatically";
+      String leaveMsg = "Safe mode will be turned off automatically";
       if(reached < 0)
         return "Safe mode is OFF.";
       if(isManual()) {
         if(getDistributedUpgradeState())
-          return autoOffMsg + " upon completion of " + 
+          return leaveMsg + " upon completion of " + 
             "the distributed upgrade: upgrade progress = " + 
             getDistributedUpgradeStatus() + "%";
-        return "Use \"hadoop dfs -safemode leave\" to turn safe mode off.";
+        leaveMsg = "Use \"hadoop dfs -safemode leave\" to turn safe mode off";
       }
+      if(blockTotal < 0)
+        return leaveMsg + ".";
       String safeBlockRatioMsg = 
         String.format("The ratio of reported blocks %.4f has " +
           (reached == 0 ? "not " : "") + "reached the threshold %.4f. ",
-          getSafeBlockRatio(), threshold) + autoOffMsg;
-      if(reached == 0)  // threshold is not reached 
+          getSafeBlockRatio(), threshold) + leaveMsg;
+      if(reached == 0 || isManual())  // threshold is not reached or manual
         return safeBlockRatioMsg + ".";
       // extension period is in progress
       return safeBlockRatioMsg + " in " 
@@ -4171,13 +4180,15 @@
    * @throws IOException
    */
   synchronized void enterSafeMode() throws IOException {
-    if (isInSafeMode()) {
-      NameNode.stateChangeLog.info("STATE* Safe mode is already ON."); 
+    if (!isInSafeMode()) {
+      safeMode = new SafeModeInfo();
       return;
     }
-    safeMode = new SafeModeInfo();
+    safeMode.setManual();
+    NameNode.stateChangeLog.info("STATE* Safe mode is ON. " 
+                                + safeMode.getTurnOffTip());
   }
-    
+
   /**
    * Leave safe mode.
    * @throws IOException

Added: hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestSafeMode.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestSafeMode.java?rev=728195&view=auto
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestSafeMode.java (added)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestSafeMode.java Fri Dec 19 16:33:14 2008
@@ -0,0 +1,95 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+
+import junit.framework.TestCase;
+
+/**
+ * This test makes sure that if SafeMode is manually entered, NameNode does not
+ * come out of safe mode even after the startup safemode conditions are met.
+ */
+public class TestSafeMode extends TestCase {
+  
+  static Log LOG = LogFactory.getLog(TestSafeMode.class);
+  
+  public void testManualSafeMode() throws IOException {
+    MiniDFSCluster cluster = null;
+    FileSystem fs = null;
+    try {
+      Configuration conf = new Configuration();
+      // disable safemode extension to make the test run faster.
+      conf.set("dfs.safemode.extension", "1");
+      cluster = new MiniDFSCluster(conf, 1, true, null);
+      cluster.waitActive();
+      
+      fs = cluster.getFileSystem();
+      Path file1 = new Path("/tmp/testManualSafeMode/file1");
+      Path file2 = new Path("/tmp/testManualSafeMode/file2");
+      
+      LOG.info("Created file1 and file2.");
+      
+      // create two files with one block each.
+      DFSTestUtil.createFile(fs, file1, 1000, (short)1, 0);
+      DFSTestUtil.createFile(fs, file2, 2000, (short)1, 0);    
+      cluster.shutdown();
+      
+      // now bring up just the NameNode.
+      cluster = new MiniDFSCluster(conf, 0, false, null);
+      cluster.waitActive();
+      
+      LOG.info("Restarted cluster with just the NameNode");
+      
+      NameNode namenode = cluster.getNameNode();
+      
+      assertTrue("No datanode is started. Should be in SafeMode", 
+                 namenode.isInSafeMode());
+      
+      // manually set safemode.
+      namenode.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      
+      // now bring up the datanode and wait for it to be active.
+      cluster.startDataNodes(conf, 1, true, null, null);
+      cluster.waitActive();
+      
+      LOG.info("Datanode is started.");
+      
+      try {
+        Thread.sleep(2000);
+      } catch (InterruptedException ignored) {}
+      
+      assertTrue("should still be in SafeMode", namenode.isInSafeMode());
+      
+      namenode.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      assertFalse("should not be in SafeMode", namenode.isInSafeMode());
+    } finally {
+      if(fs != null) fs.close();
+      if(cluster!= null) cluster.shutdown();
+    }
+  }
+}