Skip to content

Commit

Permalink
HDFS-7917. Use file to replace data dirs in test to simulate a disk f…
Browse files Browse the repository at this point in the history
…ailure. Contributed by Lei (Eddy) Xu.
  • Loading branch information
cnauroth committed Mar 23, 2015
1 parent 972f1f1 commit 2c238ae
Show file tree
Hide file tree
Showing 6 changed files with 88 additions and 70 deletions.
3 changes: 3 additions & 0 deletions hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -774,6 +774,9 @@ Release 2.7.0 - UNRELEASED

HDFS-7962. Remove duplicated logs in BlockManager. (yliu)

HDFS-7917. Use file to replace data dirs in test to simulate a disk failure.
(Lei (Eddy) Xu via cnauroth)

OPTIMIZATIONS

HDFS-7454. Reduce memory footprint for AclEntries in NameNode.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,9 @@
* Utility class for accessing package-private DataNode information during tests.
*
*/
public class DataNodeTestUtils {
public class DataNodeTestUtils {
private static final String DIR_FAILURE_SUFFIX = ".origin";

public static DatanodeRegistration
getDNRegistrationForBP(DataNode dn, String bpid) throws IOException {
return dn.getDNRegistrationForBP(bpid);
Expand Down Expand Up @@ -159,4 +161,61 @@ public static ReplicaInfo fetchReplicaInfo(final DataNode dn,
final String bpid, final long blkId) {
return FsDatasetTestUtil.fetchReplicaInfo(dn.getFSDataset(), bpid, blkId);
}

/**
* It injects disk failures to data dirs by replacing these data dirs with
* regular files.
*
* @param dirs data directories.
* @throws IOException on I/O error.
*/
public static void injectDataDirFailure(File... dirs) throws IOException {
for (File dir : dirs) {
File renamedTo = new File(dir.getPath() + DIR_FAILURE_SUFFIX);
if (renamedTo.exists()) {
throw new IOException(String.format(
"Can not inject failure to dir: %s because %s exists.",
dir, renamedTo));
}
if (!dir.renameTo(renamedTo)) {
throw new IOException(String.format("Failed to rename %s to %s.",
dir, renamedTo));
}
if (!dir.createNewFile()) {
throw new IOException(String.format(
"Failed to create file %s to inject disk failure.", dir));
}
}
}

/**
* Restore the injected data dir failures.
*
* @see {@link #injectDataDirFailures}.
* @param dirs data directories.
* @throws IOException
*/
public static void restoreDataDirFromFailure(File... dirs)
throws IOException {
for (File dir : dirs) {
File renamedDir = new File(dir.getPath() + DIR_FAILURE_SUFFIX);
if (renamedDir.exists()) {
if (dir.exists()) {
if (!dir.isFile()) {
throw new IOException(
"Injected failure data dir is supposed to be file: " + dir);
}
if (!dir.delete()) {
throw new IOException(
"Failed to delete injected failure data dir: " + dir);
}
}
if (!renamedDir.renameTo(dir)) {
throw new IOException(String.format(
"Failed to recover injected failure data dir %s to %s.",
renamedDir, dir));
}
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.BlockMissingException;
import org.apache.hadoop.hdfs.DFSConfigKeys;
Expand Down Expand Up @@ -682,26 +681,18 @@ public void testDirectlyReloadAfterCheckDiskError()
failedVolume != null);
long used = failedVolume.getDfsUsed();

try {
assertTrue("Couldn't chmod local vol: " + dirToFail,
FileUtil.setExecutable(dirToFail, false));
// Call and wait DataNode to detect disk failure.
long lastDiskErrorCheck = dn.getLastDiskErrorCheck();
dn.checkDiskErrorAsync();
while (dn.getLastDiskErrorCheck() == lastDiskErrorCheck) {
Thread.sleep(100);
}

createFile(new Path("/test1"), 32, (short)2);
assertEquals(used, failedVolume.getDfsUsed());
} finally {
// Need to restore the mode on dirToFail. Otherwise, if an Exception
// is thrown above, the following tests can not delete this data directory
// and thus fail to start MiniDFSCluster.
assertTrue("Couldn't restore executable for: " + dirToFail,
FileUtil.setExecutable(dirToFail, true));
DataNodeTestUtils.injectDataDirFailure(dirToFail);
// Call and wait DataNode to detect disk failure.
long lastDiskErrorCheck = dn.getLastDiskErrorCheck();
dn.checkDiskErrorAsync();
while (dn.getLastDiskErrorCheck() == lastDiskErrorCheck) {
Thread.sleep(100);
}

createFile(new Path("/test1"), 32, (short)2);
assertEquals(used, failedVolume.getDfsUsed());

DataNodeTestUtils.restoreDataDirFromFailure(dirToFail);
dn.reconfigurePropertyImpl(DFS_DATANODE_DATA_DIR_KEY, oldDataDir);

createFile(new Path("/test2"), 32, (short)2);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -121,10 +121,6 @@ public void tearDown() throws Exception {
if(cluster != null) {
cluster.shutdown();
}
for (int i = 0; i < 3; i++) {
FileUtil.setExecutable(new File(dataDir, "data"+(2*i+1)), true);
FileUtil.setExecutable(new File(dataDir, "data"+(2*i+2)), true);
}
}

/*
Expand Down Expand Up @@ -159,7 +155,7 @@ public void testVolumeFailure() throws Exception {
!deteteBlocks(failedDir)
) {
throw new IOException("Could not delete hdfs directory '" + failedDir + "'");
}
}
data_fail.setReadOnly();
failedDir.setReadOnly();
System.out.println("Deleteing " + failedDir.getPath() + "; exist=" + failedDir.exists());
Expand Down Expand Up @@ -217,7 +213,7 @@ public void testFailedVolumeBeingRemovedFromDataNode()
DFSTestUtil.waitReplication(fs, file1, (short) 2);

File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
assertTrue(FileUtil.setExecutable(dn0Vol1, false));
DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
DataNode dn0 = cluster.getDataNodes().get(0);
long lastDiskErrorCheck = dn0.getLastDiskErrorCheck();
dn0.checkDiskErrorAsync();
Expand Down Expand Up @@ -291,8 +287,7 @@ public void testUnderReplicationAfterVolFailure() throws Exception {
// Fail the first volume on both datanodes
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 file2 = new Path("/test2");
DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
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;
Expand Down Expand Up @@ -87,19 +86,6 @@ public void setUp() throws Exception {

@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();
Expand Down Expand Up @@ -141,8 +127,7 @@ public void testSuccessiveVolumeFailures() throws Exception {
* 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
Expand Down Expand Up @@ -179,7 +164,7 @@ public void testSuccessiveVolumeFailures() throws Exception {
* 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);
Expand Down Expand Up @@ -208,7 +193,7 @@ public void testSuccessiveVolumeFailures() throws Exception {
* 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);
Expand All @@ -233,10 +218,8 @@ public void testSuccessiveVolumeFailures() throws Exception {
* 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");
Expand Down Expand Up @@ -275,8 +258,7 @@ public void testVolFailureStatsPreservedOnNNRestart() throws Exception {
// 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);
Expand Down Expand Up @@ -323,14 +305,7 @@ public void testMultipleVolFailuresOnNode() throws Exception {

// 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.
Expand Down Expand Up @@ -380,8 +355,8 @@ public void testDataNodeReconfigureWithVolumeFailures() throws Exception {
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);
Expand Down Expand Up @@ -449,8 +424,7 @@ public void testDataNodeReconfigureWithVolumeFailures() throws Exception {

// 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);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,10 +76,6 @@ public void setUp() throws Exception {

@After
public void tearDown() throws Exception {
for (int i = 0; i < 3; i++) {
FileUtil.setExecutable(new File(dataDir, "data"+(2*i+1)), true);
FileUtil.setExecutable(new File(dataDir, "data"+(2*i+2)), true);
}
cluster.shutdown();
}

Expand Down Expand Up @@ -152,7 +148,7 @@ public void testConfigureMinValidVolumes() throws Exception {

// Fail a volume on the 2nd DN
File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
DataNodeTestUtils.injectDataDirFailure(dn2Vol1);

// Should only get two replicas (the first DN and the 3rd)
Path file1 = new Path("/test1");
Expand All @@ -165,7 +161,7 @@ public void testConfigureMinValidVolumes() throws Exception {

// If we restore the volume we should still only be able to get
// two replicas since the DN is still considered dead.
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, true));
DataNodeTestUtils.restoreDataDirFromFailure(dn2Vol1);
Path file2 = new Path("/test2");
DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L);
DFSTestUtil.waitReplication(fs, file2, (short)2);
Expand Down

0 comments on commit 2c238ae

Please sign in to comment.