You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by el...@apache.org on 2010/11/18 22:09:21 UTC
svn commit: r1036631 - in /hadoop/hdfs/trunk: ./ src/java/
src/java/org/apache/hadoop/hdfs/
src/java/org/apache/hadoop/hdfs/server/namenode/
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/
Author: eli
Date: Thu Nov 18 21:09:21 2010
New Revision: 1036631
URL: http://svn.apache.org/viewvc?rev=1036631&view=rev
Log:
HDFS-528. Add ability for safemode to wait for a minimum number of live datanodes. Contributed by Todd Lipcon
Added:
hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestSafeMode.java
Modified:
hadoop/hdfs/trunk/CHANGES.txt
hadoop/hdfs/trunk/src/java/hdfs-default.xml
hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=1036631&r1=1036630&r2=1036631&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Thu Nov 18 21:09:21 2010
@@ -62,6 +62,9 @@ Release 0.22.0 - Unreleased
HDFS-895. Allow hflush/sync to occur in parallel with new writes
to the file. (Todd Lipcon via hairong)
+ HDFS-528. Add ability for safemode to wait for a minimum number of
+ live datanodes (Todd Lipcon via eli)
+
IMPROVEMENTS
HDFS-1304. Add a new unit test for HftpFileSystem.open(..). (szetszwo)
Modified: hadoop/hdfs/trunk/src/java/hdfs-default.xml
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/hdfs-default.xml?rev=1036631&r1=1036630&r2=1036631&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/hdfs-default.xml (original)
+++ hadoop/hdfs/trunk/src/java/hdfs-default.xml Thu Nov 18 21:09:21 2010
@@ -344,12 +344,27 @@ creations/deletions), or "all".</descrip
<description>
Specifies the percentage of blocks that should satisfy
the minimal replication requirement defined by dfs.namenode.replication.min.
- Values less than or equal to 0 mean not to start in safe mode.
+ Values less than or equal to 0 mean not to wait for any particular
+ percentage of blocks before exiting safemode.
Values greater than 1 will make safe mode permanent.
</description>
</property>
<property>
+ <name>dfs.namenode.safemode.min.datanodes</name>
+ <value>0</value>
+ <description>
+ Specifies the number of datanodes that must be considered alive
+ before the name node exits safemode.
+ Values less than or equal to 0 mean not to take the number of live
+ datanodes into account when deciding whether to remain in safe mode
+ during startup.
+ Values greater than the number of datanodes in the cluster
+ will make safe mode permanent.
+ </description>
+</property>
+
+<property>
<name>dfs.namenode.safemode.extension</name>
<value>30000</value>
<description>
Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java?rev=1036631&r1=1036630&r2=1036631&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java Thu Nov 18 21:09:21 2010
@@ -57,6 +57,8 @@ public class DFSConfigKeys extends Commo
public static final int DFS_NAMENODE_SAFEMODE_EXTENSION_DEFAULT = 30000;
public static final String DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY = "dfs.namenode.safemode.threshold-pct";
public static final float DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_DEFAULT = 0.999f;
+ public static final String DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY = "dfs.namenode.safemode.min.datanodes";
+ public static final int DFS_NAMENODE_SAFEMODE_MIN_DATANODES_DEFAULT = 0;
public static final String DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY = "dfs.namenode.secondary.http-address";
public static final String DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_DEFAULT = "0.0.0.0:50090";
public static final String DFS_NAMENODE_CHECKPOINT_PERIOD_KEY = "dfs.namenode.checkpoint.period";
Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1036631&r1=1036630&r2=1036631&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Thu Nov 18 21:09:21 2010
@@ -2595,6 +2595,10 @@ public class FSNamesystem implements FSC
// no need to update its timestamp
// because its is done when the descriptor is created
}
+
+ if (safeMode != null) {
+ safeMode.checkMode();
+ }
return;
} finally {
writeUnlock();
@@ -2929,6 +2933,10 @@ public class FSNamesystem implements FSC
}
unprotectedRemoveDatanode(nodeInfo);
clusterMap.remove(nodeInfo);
+
+ if (safeMode != null) {
+ safeMode.checkMode();
+ }
}
void unprotectedRemoveDatanode(DatanodeDescriptor nodeDescr) {
@@ -3730,6 +3738,8 @@ public class FSNamesystem implements FSC
// configuration fields
/** Safe mode threshold condition %.*/
private double threshold;
+ /** Safe mode minimum number of datanodes alive */
+ private int datanodeThreshold;
/** Safe mode extension after the threshold. */
private int extension;
/** Min replication required by safe mode. */
@@ -3759,6 +3769,9 @@ public class FSNamesystem implements FSC
*/
SafeModeInfo(Configuration conf) {
this.threshold = conf.getFloat(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 0.95f);
+ this.datanodeThreshold = conf.getInt(
+ DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY,
+ DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_DEFAULT);
this.extension = conf.getInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
this.safeReplication = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
@@ -3776,6 +3789,7 @@ public class FSNamesystem implements FSC
*/
private SafeModeInfo() {
this.threshold = 1.5f; // this threshold can never be reached
+ this.datanodeThreshold = Integer.MAX_VALUE;
this.extension = Integer.MAX_VALUE;
this.safeReplication = Short.MAX_VALUE + 1; // more than maxReplication
this.blockTotal = -1;
@@ -3868,7 +3882,8 @@ public class FSNamesystem implements FSC
* if DFS is empty or {@link #threshold} == 0
*/
boolean needEnter() {
- return threshold != 0 && blockSafe < blockThreshold;
+ return (threshold != 0 && blockSafe < blockThreshold) ||
+ (getNumLiveDataNodes() < datanodeThreshold);
}
/**
@@ -3958,17 +3973,37 @@ public class FSNamesystem implements FSC
}
if(blockTotal < 0)
return leaveMsg + ".";
-
- String msg = null;
+
+ int numLive = getNumLiveDataNodes();
+ String msg = "";
if (reached == 0) {
- msg = String.format("The reported blocks %d needs additional %d"
- + " blocks to reach the threshold %.4f of total blocks %d. %s",
- blockSafe, (blockThreshold - blockSafe), threshold, blockTotal,
- leaveMsg);
+ if (blockSafe < blockThreshold) {
+ msg += String.format(
+ "The reported blocks %d needs additional %d"
+ + " blocks to reach the threshold %.4f of total blocks %d.",
+ blockSafe, (blockThreshold - blockSafe), threshold, blockTotal);
+ }
+ if (numLive < datanodeThreshold) {
+ if (!"".equals(msg)) {
+ msg += "\n";
+ }
+ msg += String.format(
+ "The number of live datanodes %d needs an additional %d live "
+ + "datanodes to reach the minimum number %d.",
+ numLive, datanodeThreshold - numLive, datanodeThreshold);
+ }
+ msg += " " + leaveMsg;
} else {
msg = String.format("The reported blocks %d has reached the threshold"
- + " %.4f of total blocks %d. %s", blockSafe, threshold,
- blockTotal, leaveMsg);
+ + " %.4f of total blocks %d.", blockSafe, threshold,
+ blockTotal);
+
+ if (datanodeThreshold > 0) {
+ msg += String.format(" The number of live datanodes %d has reached "
+ + "the minimum number %d.",
+ numLive, datanodeThreshold);
+ }
+ msg += " " + leaveMsg;
}
if(reached == 0 || isManual()) { // threshold is not reached or manual
return msg + ".";
Added: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestSafeMode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestSafeMode.java?rev=1036631&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestSafeMode.java (added)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestSafeMode.java Thu Nov 18 21:09:21 2010
@@ -0,0 +1,159 @@
+/**
+ * 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.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.FSConstants.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.apache.hadoop.hdfs.DFSTestUtil;
+import junit.framework.TestCase;
+
+/**
+ * Tests to verify safe mode correctness.
+ */
+public class TestSafeMode extends TestCase {
+
+ static Log LOG = LogFactory.getLog(TestSafeMode.class);
+
+ /**
+ * This test verifies that if SafeMode is manually entered, name-node does not
+ * come out of safe mode even after the startup safe mode conditions are met.
+ * <ol>
+ * <li>Start cluster with 1 data-node.</li>
+ * <li>Create 2 files with replication 1.</li>
+ * <li>Re-start cluster with 0 data-nodes.
+ * Name-node should stay in automatic safe-mode.</li>
+ * <li>Enter safe mode manually.</li>
+ * <li>Start the data-node.</li>
+ * <li>Wait longer than <tt>dfs.safemode.extension</tt> and
+ * verify that the name-node is still in safe mode.</li>
+ * </ol>
+ *
+ * @throws IOException
+ */
+ public void testManualSafeMode() throws IOException {
+ MiniDFSCluster cluster = null;
+ DistributedFileSystem fs = null;
+ try {
+ Configuration conf = new HdfsConfiguration();
+ // disable safemode extension to make the test run faster.
+ conf.set(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, "1");
+ cluster = new MiniDFSCluster.Builder(conf).build();
+ cluster.waitActive();
+
+ fs = (DistributedFileSystem)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);
+ fs.close();
+ cluster.shutdown();
+
+ // now bring up just the NameNode.
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(false).build();
+ cluster.waitActive();
+ fs = (DistributedFileSystem)cluster.getFileSystem();
+
+ LOG.info("Restarted cluster with just the NameNode");
+
+ assertTrue("No datanode is started. Should be in SafeMode",
+ fs.setSafeMode(SafeModeAction.SAFEMODE_GET));
+
+ // manually set safemode.
+ fs.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.");
+
+ // wait longer than dfs.safemode.extension
+ try {
+ Thread.sleep(2000);
+ } catch (InterruptedException ignored) {}
+
+ assertTrue("should still be in SafeMode",
+ fs.setSafeMode(SafeModeAction.SAFEMODE_GET));
+
+ fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+ assertFalse("should not be in SafeMode",
+ fs.setSafeMode(SafeModeAction.SAFEMODE_GET));
+ } finally {
+ if(fs != null) fs.close();
+ if(cluster!= null) cluster.shutdown();
+ }
+ }
+
+
+ /**
+ * Verify that the NameNode stays in safemode when dfs.safemode.datanode.min
+ * is set to a number greater than the number of live datanodes.
+ */
+ public void testDatanodeThreshold() throws IOException {
+ MiniDFSCluster cluster = null;
+ DistributedFileSystem fs = null;
+ try {
+ Configuration conf = new Configuration();
+ 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 1 live"));
+
+ // 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();
+ }
+ }
+}