|
@@ -34,7 +34,6 @@ import org.apache.commons.logging.impl.Log4JLogger;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.conf.ReconfigurationException;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
-import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
@@ -87,19 +86,6 @@ public class TestDataNodeVolumeFailureReporting {
|
|
|
|
|
|
@After
|
|
|
public void tearDown() throws Exception {
|
|
|
- // Restore executable permission on all directories where a failure may have
|
|
|
- // been simulated by denying execute access. This is based on the maximum
|
|
|
- // number of datanodes and the maximum number of storages per data node used
|
|
|
- // throughout the tests in this suite.
|
|
|
- assumeTrue(!Path.WINDOWS);
|
|
|
- int maxDataNodes = 3;
|
|
|
- int maxStoragesPerDataNode = 4;
|
|
|
- for (int i = 0; i < maxDataNodes; i++) {
|
|
|
- for (int j = 1; j <= maxStoragesPerDataNode; j++) {
|
|
|
- String subDir = "data" + ((i * maxStoragesPerDataNode) + j);
|
|
|
- FileUtil.setExecutable(new File(dataDir, subDir), true);
|
|
|
- }
|
|
|
- }
|
|
|
IOUtils.cleanup(LOG, fs);
|
|
|
if (cluster != null) {
|
|
|
cluster.shutdown();
|
|
@@ -141,8 +127,7 @@ public class TestDataNodeVolumeFailureReporting {
|
|
|
* fail. The client does not retry failed nodes even though
|
|
|
* perhaps they could succeed because just a single volume failed.
|
|
|
*/
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, false));
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn1Vol1, dn2Vol1);
|
|
|
|
|
|
/*
|
|
|
* Create file1 and wait for 3 replicas (ie all DNs can still
|
|
@@ -179,7 +164,7 @@ public class TestDataNodeVolumeFailureReporting {
|
|
|
* Now fail a volume on the third datanode. We should be able to get
|
|
|
* three replicas since we've already identified the other failures.
|
|
|
*/
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol1, false));
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn3Vol1);
|
|
|
Path file2 = new Path("/test2");
|
|
|
DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L);
|
|
|
DFSTestUtil.waitReplication(fs, file2, (short)3);
|
|
@@ -208,7 +193,7 @@ public class TestDataNodeVolumeFailureReporting {
|
|
|
* and that it's no longer up. Only wait for two replicas since
|
|
|
* we'll never get a third.
|
|
|
*/
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol2, false));
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn3Vol2);
|
|
|
Path file3 = new Path("/test3");
|
|
|
DFSTestUtil.createFile(fs, file3, 1024, (short)3, 1L);
|
|
|
DFSTestUtil.waitReplication(fs, file3, (short)2);
|
|
@@ -233,10 +218,8 @@ public class TestDataNodeVolumeFailureReporting {
|
|
|
* restart, so file creation should be able to succeed after
|
|
|
* restoring the data directories and restarting the datanodes.
|
|
|
*/
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, true));
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, true));
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol1, true));
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol2, true));
|
|
|
+ DataNodeTestUtils.restoreDataDirFromFailure(
|
|
|
+ dn1Vol1, dn2Vol1, dn3Vol1, dn3Vol2);
|
|
|
cluster.restartDataNodes();
|
|
|
cluster.waitActive();
|
|
|
Path file4 = new Path("/test4");
|
|
@@ -275,8 +258,7 @@ public class TestDataNodeVolumeFailureReporting {
|
|
|
// third healthy so one node in the pipeline will not fail).
|
|
|
File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
|
|
|
File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, false));
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn1Vol1, dn2Vol1);
|
|
|
|
|
|
Path file1 = new Path("/test1");
|
|
|
DFSTestUtil.createFile(fs, file1, 1024, (short)2, 1L);
|
|
@@ -323,14 +305,7 @@ public class TestDataNodeVolumeFailureReporting {
|
|
|
|
|
|
// Make the first two volume directories on the first two datanodes
|
|
|
// non-accessible.
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1,
|
|
|
- false));
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol2,
|
|
|
- false));
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1,
|
|
|
- false));
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol2,
|
|
|
- false));
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn1Vol1, dn1Vol2, dn2Vol1, dn2Vol2);
|
|
|
|
|
|
// Create file1 and wait for 3 replicas (ie all DNs can still store a block).
|
|
|
// Then assert that all DNs are up, despite the volume failures.
|
|
@@ -380,8 +355,8 @@ public class TestDataNodeVolumeFailureReporting {
|
|
|
File dn1Vol2 = new File(dataDir, "data"+(2*0+2));
|
|
|
File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
|
|
|
File dn2Vol2 = new File(dataDir, "data"+(2*1+2));
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, false));
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn1Vol1);
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn2Vol1);
|
|
|
|
|
|
Path file1 = new Path("/test1");
|
|
|
DFSTestUtil.createFile(fs, file1, 1024, (short)2, 1L);
|
|
@@ -449,8 +424,7 @@ public class TestDataNodeVolumeFailureReporting {
|
|
|
|
|
|
// Replace failed volume with healthy volume and run reconfigure DataNode.
|
|
|
// The failed volume information should be cleared.
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, true));
|
|
|
- assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, true));
|
|
|
+ DataNodeTestUtils.restoreDataDirFromFailure(dn1Vol1, dn2Vol1);
|
|
|
reconfigureDataNode(dns.get(0), dn1Vol1, dn1Vol2);
|
|
|
reconfigureDataNode(dns.get(1), dn2Vol1, dn2Vol2);
|
|
|
|