Explorar o código

HDFS-2817. Combine the two TestSafeMode test suites. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1234221 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon %!s(int64=13) %!d(string=hai) anos
pai
achega
c840548d6b

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

@@ -272,6 +272,8 @@ Release 0.23.1 - UNRELEASED
     HDFS-2803. Add logging to LeaseRenewer for better lease expiration debugging.
     (Jimmy Xiang via todd)
 
+    HDFS-2817. Combine the two TestSafeMode test suites. (todd)
+
   OPTIMIZATIONS
 
     HDFS-2130. Switch default checksum to CRC32C. (todd)

+ 48 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java

@@ -113,6 +113,21 @@ public class TestSafeMode {
         dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE));
   }
 
+  /**
+   * Test that, if there are no blocks in the filesystem,
+   * the NameNode doesn't enter the "safemode extension" period.
+   */
+  @Test(timeout=45000)
+  public void testNoExtensionIfNoBlocks() throws IOException {
+    cluster.getConfiguration(0).setInt(
+        DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 60000);
+    cluster.restartNameNode();
+    // Even though we have safemode extension set high, we should immediately
+    // exit safemode on startup because there are no blocks in the namespace.
+    String status = cluster.getNameNode().getNamesystem().getSafemode();
+    assertEquals("", status);
+  }
+
   public interface FSRun {
     public abstract void run(FileSystem fs) throws IOException;
   }
@@ -193,5 +208,37 @@ public class TestSafeMode {
     assertFalse("Could not leave SM",
         dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE));
   }
-  
+
+  /**
+   * Verify that the NameNode stays in safemode when dfs.safemode.datanode.min
+   * is set to a number greater than the number of live datanodes.
+   */
+  @Test
+  public void testDatanodeThreshold() throws IOException {
+    cluster.shutdown();
+    Configuration conf = cluster.getConfiguration(0);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 1);
+
+    cluster.restartNameNode();
+    fs = (DistributedFileSystem)cluster.getFileSystem();
+
+    String tipMsg = cluster.getNamesystem().getSafemode();
+    assertTrue("Safemode tip message looks right: " + tipMsg,
+               tipMsg.contains("The number of live datanodes 0 needs an additional " +
+                               "2 live datanodes to reach the minimum number 1. " +
+                               "Safe mode will be turned off automatically."));
+
+    // Start a datanode
+    cluster.startDataNodes(conf, 1, true, null, null);
+
+    // Wait long enough for safemode check to refire
+    try {
+      Thread.sleep(1000);
+    } catch (InterruptedException ignored) {}
+
+    // We now should be out of safe mode.
+    assertEquals("", cluster.getNamesystem().getSafemode());
+  }
+
 }

+ 0 - 82
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSafeMode.java

@@ -1,82 +0,0 @@
-/**
- * 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.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-
-import org.junit.Test;
-import static org.junit.Assert.*;
-
-/**
- * Tests to verify safe mode correctness.
- */
-public class TestSafeMode {
-  
-  /**
-   * Verify that the NameNode stays in safemode when dfs.safemode.datanode.min
-   * is set to a number greater than the number of live datanodes.
-   */
-  @Test
-  public void testDatanodeThreshold() throws IOException {
-    MiniDFSCluster cluster = null;
-    DistributedFileSystem fs = null;
-    try {
-      Configuration conf = new HdfsConfiguration();
-      conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
-      conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 1);
-
-      // bring up a cluster with no datanodes
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(true).build();
-      cluster.waitActive();
-      fs = (DistributedFileSystem)cluster.getFileSystem();
-
-      assertTrue("No datanode started, but we require one - safemode expected",
-                 fs.setSafeMode(SafeModeAction.SAFEMODE_GET));
-
-      String tipMsg = cluster.getNamesystem().getSafeModeTip();
-      assertTrue("Safemode tip message looks right",
-                 tipMsg.contains("The number of live datanodes 0 needs an additional " +
-                                 "2 live datanodes to reach the minimum number 1. " +
-                                 "Safe mode will be turned off automatically."));
-
-      // Start a datanode
-      cluster.startDataNodes(conf, 1, true, null, null);
-
-      // Wait long enough for safemode check to refire
-      try {
-        Thread.sleep(1000);
-      } catch (InterruptedException ignored) {}
-
-      // We now should be out of safe mode.
-      assertFalse(
-        "Out of safe mode after starting datanode.",
-        fs.setSafeMode(SafeModeAction.SAFEMODE_GET));
-    } finally {
-      if (fs != null) fs.close();
-      if (cluster != null) cluster.shutdown();
-    }
-  }
-}