hadoop-hdfs-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From e..@apache.org
Subject svn commit: r1156974 - in /hadoop/common/trunk/hdfs: CHANGES.txt src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
Date Fri, 12 Aug 2011 06:09:35 GMT
Author: eli
Date: Fri Aug 12 06:09:35 2011
New Revision: 1156974

URL: http://svn.apache.org/viewvc?rev=1156974&view=rev
Log:
HDFS-2186. DN volume failures on startup are not counted. Contributed by Eli Collins

Modified:
    hadoop/common/trunk/hdfs/CHANGES.txt
    hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
    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=1156974&r1=1156973&r2=1156974&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hdfs/CHANGES.txt Fri Aug 12 06:09:35 2011
@@ -959,6 +959,8 @@ Trunk (unreleased changes)
 
     HDFS-2235. Encode servlet paths. (eli)
 
+    HDFS-2186. DN volume failures on startup are not counted. (eli)
+
   BREAKDOWN OF HDFS-1073 SUBTASKS
 
     HDFS-1521. Persist transaction ID on disk between NN restarts.

Modified: hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java?rev=1156974&r1=1156973&r2=1156974&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
(original)
+++ hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
Fri Aug 12 06:09:35 2011
@@ -775,12 +775,13 @@ public class FSDataset implements FSData
      */
     private volatile List<FSVolume> volumes = null;
     BlockVolumeChoosingPolicy blockChooser;
-    int numFailedVolumes = 0;
+    int numFailedVolumes;
 
-    FSVolumeSet(FSVolume[] volumes, BlockVolumeChoosingPolicy blockChooser) {
+    FSVolumeSet(FSVolume[] volumes, int failedVols, BlockVolumeChoosingPolicy blockChooser)
{
       List<FSVolume> list = Arrays.asList(volumes);
       this.volumes = Collections.unmodifiableList(list);
       this.blockChooser = blockChooser;
+      this.numFailedVolumes = failedVols;
     }
     
     private int numberOfVolumes() {
@@ -1144,15 +1145,19 @@ public class FSDataset implements FSData
     String[] dataDirs = conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
 
     int volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
-
+    int volsFailed = volsConfigured - storage.getNumStorageDirs();
     this.validVolsRequired = volsConfigured - volFailuresTolerated;
 
-    if (validVolsRequired < 1
-        || validVolsRequired > storage.getNumStorageDirs()) {
+    if (volFailuresTolerated < 0 || volFailuresTolerated >= volsConfigured) {
+      throw new DiskErrorException("Invalid volume failure "
+          + " config value: " + volFailuresTolerated);
+    }
+    if (volsFailed > volFailuresTolerated) {
       throw new DiskErrorException("Too many failed volumes - "
           + "current valid volumes: " + storage.getNumStorageDirs() 
           + ", volumes configured: " + volsConfigured 
-          + ", volume failures tolerated: " + volFailuresTolerated );
+          + ", volumes failed: " + volsFailed
+          + ", volume failures tolerated: " + volFailuresTolerated);
     }
 
     FSVolume[] volArray = new FSVolume[storage.getNumStorageDirs()];
@@ -1170,7 +1175,7 @@ public class FSDataset implements FSData
             RoundRobinVolumesPolicy.class,
             BlockVolumeChoosingPolicy.class),
         conf);
-    volumes = new FSVolumeSet(volArray, blockChooserImpl);
+    volumes = new FSVolumeSet(volArray, volsFailed, blockChooserImpl);
     volumes.getVolumeMap(volumeMap);
 
     File[] roots = new File[storage.getNumStorageDirs()];

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=1156974&r1=1156973&r2=1156974&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 Aug 12 06:09:35 2011
@@ -24,6 +24,7 @@ import static org.junit.Assume.assumeTru
 
 import java.io.File;
 import java.io.IOException;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -37,6 +38,7 @@ import org.apache.hadoop.hdfs.DFSTestUti
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Before;
@@ -189,7 +191,7 @@ public class TestDataNodeVolumeFailureTo
    */
   private void restartDatanodes(int volTolerated, boolean manageDfsDirs)
       throws IOException {
-    //Make sure no datanode is running
+    // Make sure no datanode is running
     cluster.shutdownDataNodes();
     conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, volTolerated);
     cluster.startDataNodes(conf, 1, manageDfsDirs, null, null);
@@ -226,7 +228,7 @@ public class TestDataNodeVolumeFailureTo
    */
   private void testVolumeConfig(int volumesTolerated, int volumesFailed,
       boolean expectedBPServiceState, boolean manageDfsDirs)
-      throws IOException, InterruptedException {
+      throws IOException, InterruptedException, TimeoutException {
     assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
     final int dnIndex = 0;
     // Fail the current directory since invalid storage directory perms
@@ -261,4 +263,30 @@ public class TestDataNodeVolumeFailureTo
     assertEquals("Couldn't chmod local vol", 0,
         FileUtil.chmod(dir.toString(), "000"));
   }
+
+  /**
+   * Test that a volume that is considered failed on startup is seen as
+   *  a failed volume by the NN.
+   */
+  @Test
+  public void testFailedVolumeOnStartupIsCounted() throws Exception {
+    assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
+    final DatanodeManager dm = cluster.getNamesystem().getBlockManager(
+    ).getDatanodeManager();
+    long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(dm);
+    File dir = new File(MiniDFSCluster.getStorageDir(0, 0), "current");
+
+    try {
+      prepareDirToFail(dir);
+      restartDatanodes(1, false);
+      // The cluster is up..
+      assertEquals(true, cluster.getDataNodes().get(0)
+          .isBPServiceAlive(cluster.getNamesystem().getBlockPoolId()));
+      // but there has been a single volume failure
+      DFSTestUtil.waitForDatanodeStatus(dm, 1, 0, 1,
+          origCapacity / 2, WAIT_FOR_HEARTBEATS);
+    } finally {
+      FileUtil.chmod(dir.toString(), "755");
+    }
+  }
 }



Mime
View raw message