|
@@ -34,6 +34,7 @@ import java.util.Map;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
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;
|
|
@@ -219,12 +220,7 @@ public class TestDataNodeVolumeFailure {
|
|
|
File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
|
|
|
DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
|
|
|
DataNode dn0 = cluster.getDataNodes().get(0);
|
|
|
- long lastDiskErrorCheck = dn0.getLastDiskErrorCheck();
|
|
|
- dn0.checkDiskErrorAsync();
|
|
|
- // Wait checkDiskError thread finish to discover volume failure.
|
|
|
- while (dn0.getLastDiskErrorCheck() == lastDiskErrorCheck) {
|
|
|
- Thread.sleep(100);
|
|
|
- }
|
|
|
+ checkDiskErrorSync(dn0);
|
|
|
|
|
|
// Verify dn0Vol1 has been completely removed from DN0.
|
|
|
// 1. dn0Vol1 is removed from DataStorage.
|
|
@@ -270,6 +266,102 @@ public class TestDataNodeVolumeFailure {
|
|
|
assertFalse(dataDirStrs[0].contains(dn0Vol1.getAbsolutePath()));
|
|
|
}
|
|
|
|
|
|
+ private static void checkDiskErrorSync(DataNode dn)
|
|
|
+ throws InterruptedException {
|
|
|
+ final long lastDiskErrorCheck = dn.getLastDiskErrorCheck();
|
|
|
+ dn.checkDiskErrorAsync();
|
|
|
+ // Wait 10 seconds for checkDiskError thread to finish and discover volume
|
|
|
+ // failures.
|
|
|
+ int count = 100;
|
|
|
+ while (count > 0 && dn.getLastDiskErrorCheck() == lastDiskErrorCheck) {
|
|
|
+ Thread.sleep(100);
|
|
|
+ count--;
|
|
|
+ }
|
|
|
+ assertTrue("Disk checking thread does not finish in 10 seconds",
|
|
|
+ count > 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test DataNode stops when the number of failed volumes exceeds
|
|
|
+ * dfs.datanode.failed.volumes.tolerated .
|
|
|
+ */
|
|
|
+ @Test(timeout=10000)
|
|
|
+ public void testDataNodeShutdownAfterNumFailedVolumeExceedsTolerated()
|
|
|
+ throws InterruptedException, IOException {
|
|
|
+ // make both data directories to fail on dn0
|
|
|
+ final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
|
|
|
+ final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn0Vol1, dn0Vol2);
|
|
|
+ DataNode dn0 = cluster.getDataNodes().get(0);
|
|
|
+ checkDiskErrorSync(dn0);
|
|
|
+
|
|
|
+ // DN0 should stop after the number of failure disks exceed tolerated
|
|
|
+ // value (1).
|
|
|
+ assertFalse(dn0.shouldRun());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test that DN does not shutdown, as long as failure volumes being hot swapped.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testVolumeFailureRecoveredByHotSwappingVolume()
|
|
|
+ throws InterruptedException, ReconfigurationException, IOException {
|
|
|
+ final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
|
|
|
+ final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
|
|
|
+ final DataNode dn0 = cluster.getDataNodes().get(0);
|
|
|
+ final String oldDataDirs = dn0.getConf().get(
|
|
|
+ DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
|
|
|
+
|
|
|
+ // Fail dn0Vol1 first.
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
|
|
|
+ checkDiskErrorSync(dn0);
|
|
|
+
|
|
|
+ // Hot swap out the failure volume.
|
|
|
+ String dataDirs = dn0Vol2.getPath();
|
|
|
+ dn0.reconfigurePropertyImpl(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
|
|
|
+ dataDirs);
|
|
|
+
|
|
|
+ // Fix failure volume dn0Vol1 and remount it back.
|
|
|
+ DataNodeTestUtils.restoreDataDirFromFailure(dn0Vol1);
|
|
|
+ dn0.reconfigurePropertyImpl(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
|
|
|
+ oldDataDirs);
|
|
|
+
|
|
|
+ // Fail dn0Vol2. Now since dn0Vol1 has been fixed, DN0 has sufficient
|
|
|
+ // resources, thus it should keep running.
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn0Vol2);
|
|
|
+ checkDiskErrorSync(dn0);
|
|
|
+ assertTrue(dn0.shouldRun());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test changing the number of volumes does not impact the disk failure
|
|
|
+ * tolerance.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testTolerateVolumeFailuresAfterAddingMoreVolumes()
|
|
|
+ throws InterruptedException, ReconfigurationException, IOException {
|
|
|
+ final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
|
|
|
+ final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
|
|
|
+ final File dn0VolNew = new File(dataDir, "data_new");
|
|
|
+ final DataNode dn0 = cluster.getDataNodes().get(0);
|
|
|
+ final String oldDataDirs = dn0.getConf().get(
|
|
|
+ DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
|
|
|
+
|
|
|
+ // Add a new volume to DN0
|
|
|
+ dn0.reconfigurePropertyImpl(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
|
|
|
+ oldDataDirs + "," + dn0VolNew.getAbsolutePath());
|
|
|
+
|
|
|
+ // Fail dn0Vol1 first and hot swap it.
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
|
|
|
+ checkDiskErrorSync(dn0);
|
|
|
+ assertTrue(dn0.shouldRun());
|
|
|
+
|
|
|
+ // Fail dn0Vol2, now dn0 should stop, because we only tolerate 1 disk failure.
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn0Vol2);
|
|
|
+ checkDiskErrorSync(dn0);
|
|
|
+ assertFalse(dn0.shouldRun());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Test that there are under replication blocks after vol failures
|
|
|
*/
|