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 to...@apache.org on 2011/07/08 02:16:08 UTC

svn commit: r1144100 - in /hadoop/common/trunk/hdfs: CHANGES.txt src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java

Author: todd
Date: Fri Jul  8 00:16:07 2011
New Revision: 1144100

URL: http://svn.apache.org/viewvc?rev=1144100&view=rev
Log:
HDFS-2111. Add tests for ensuring that the DN will start with a few bad data directories. Contributed by Harsh J Chouraria.

Modified:
    hadoop/common/trunk/hdfs/CHANGES.txt
    hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java

Modified: hadoop/common/trunk/hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/CHANGES.txt?rev=1144100&r1=1144099&r2=1144100&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hdfs/CHANGES.txt Fri Jul  8 00:16:07 2011
@@ -540,6 +540,9 @@ Trunk (unreleased changes)
     HDFS-2109. Store uMask as member variable to DFSClient.Conf.  (Bharath
     Mundlapudi via szetszwo)
 
+    HDFS-2111. Add tests for ensuring that the DN will start with a few bad
+    data directories. (Harsh J Chouraria via todd)
+
   OPTIMIZATIONS
 
     HDFS-1458. Improve checkpoint performance by avoiding unnecessary image

Modified: hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java?rev=1144100&r1=1144099&r2=1144100&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java (original)
+++ hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java Fri Jul  8 00:16:07 2011
@@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configurat
 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.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -95,6 +94,54 @@ public class TestDataNodeVolumeFailureTo
 
   /**
    * Test the DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY configuration
+   * option, ie the DN tolerates a failed-to-use scenario during
+   * its start-up.
+   */
+  @Test
+  public void testValidVolumesAtStartup() throws Exception {
+    assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
+
+    // Make sure no DNs are running.
+    cluster.shutdownDataNodes();
+
+    // Bring up a datanode with two default data dirs, but with one bad one.
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1);
+
+    // We use subdirectories 0 and 1 in order to have only a single
+    // data dir's parent inject a failure.
+    File tld = new File(MiniDFSCluster.getBaseDirectory(), "badData");
+    File dataDir1 = new File(tld, "data1");
+    File dataDir1Actual = new File(dataDir1, "1");
+    dataDir1Actual.mkdirs();
+    // Force an IOE to occur on one of the dfs.data.dir.
+    File dataDir2 = new File(tld, "data2");
+    prepareDirToFail(dataDir2);
+    File dataDir2Actual = new File(dataDir2, "2");
+
+    // Start one DN, with manually managed DN dir
+    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
+        dataDir1Actual.getPath() + "," + dataDir2Actual.getPath());
+    cluster.startDataNodes(conf, 1, false, null, null);
+    cluster.waitActive();
+
+    try {
+      assertTrue("The DN should have started up fine.",
+          cluster.isDataNodeUp());
+      DataNode dn = cluster.getDataNodes().get(0);
+      String si = dn.getFSDataset().getStorageInfo();
+      assertTrue("The DN should have started with this directory",
+          si.contains(dataDir1Actual.getPath()));
+      assertFalse("The DN shouldn't have a bad directory.",
+          si.contains(dataDir2Actual.getPath()));
+    } finally {
+      cluster.shutdownDataNodes();
+      FileUtil.chmod(dataDir2.toString(), "755");
+    }
+
+  }
+
+  /**
+   * Test the DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY configuration
    * option, ie the DN shuts itself down when the number of failures
    * experienced drops below the tolerated amount.
    */