|
@@ -25,16 +25,22 @@ import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertFalse;
|
|
import static org.junit.Assert.assertFalse;
|
|
import static org.junit.Assert.assertNull;
|
|
import static org.junit.Assert.assertNull;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
+import static org.junit.Assert.fail;
|
|
|
|
|
|
import java.io.File;
|
|
import java.io.File;
|
|
|
|
+import java.lang.management.ManagementFactory;
|
|
import java.net.URI;
|
|
import java.net.URI;
|
|
import java.net.URISyntaxException;
|
|
import java.net.URISyntaxException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
|
|
+import javax.management.MBeanServer;
|
|
|
|
+import javax.management.ObjectName;
|
|
|
|
+
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
+import org.apache.hadoop.conf.ReconfigurationException;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
@@ -51,6 +57,7 @@ import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.log4j.Level;
|
|
import org.apache.log4j.Level;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
|
|
+import org.junit.Assert;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Rule;
|
|
import org.junit.Rule;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
@@ -406,8 +413,8 @@ public class TestDataNodeVolumeFailureReporting {
|
|
DataNodeTestUtils.triggerHeartbeat(dns.get(0));
|
|
DataNodeTestUtils.triggerHeartbeat(dns.get(0));
|
|
DataNodeTestUtils.triggerHeartbeat(dns.get(1));
|
|
DataNodeTestUtils.triggerHeartbeat(dns.get(1));
|
|
|
|
|
|
- checkFailuresAtDataNode(dns.get(0), 1, false, dn1Vol1.getAbsolutePath());
|
|
|
|
- checkFailuresAtDataNode(dns.get(1), 1, false, dn2Vol1.getAbsolutePath());
|
|
|
|
|
|
+ checkFailuresAtDataNode(dns.get(0), 1, true, dn1Vol1.getAbsolutePath());
|
|
|
|
+ checkFailuresAtDataNode(dns.get(1), 1, true, dn2Vol1.getAbsolutePath());
|
|
|
|
|
|
// Ensure we wait a sufficient amount of time.
|
|
// Ensure we wait a sufficient amount of time.
|
|
assert (WAIT_FOR_HEARTBEATS * 10) > WAIT_FOR_DEATH;
|
|
assert (WAIT_FOR_HEARTBEATS * 10) > WAIT_FOR_DEATH;
|
|
@@ -415,9 +422,9 @@ public class TestDataNodeVolumeFailureReporting {
|
|
// The NN reports two volume failures again.
|
|
// The NN reports two volume failures again.
|
|
DFSTestUtil.waitForDatanodeStatus(dm, 3, 0, 2,
|
|
DFSTestUtil.waitForDatanodeStatus(dm, 3, 0, 2,
|
|
origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
|
|
origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
|
|
- checkAggregateFailuresAtNameNode(false, 2);
|
|
|
|
- checkFailuresAtNameNode(dm, dns.get(0), false, dn1Vol1.getAbsolutePath());
|
|
|
|
- checkFailuresAtNameNode(dm, dns.get(1), false, dn2Vol1.getAbsolutePath());
|
|
|
|
|
|
+ checkAggregateFailuresAtNameNode(true, 2);
|
|
|
|
+ checkFailuresAtNameNode(dm, dns.get(0), true, dn1Vol1.getAbsolutePath());
|
|
|
|
+ checkFailuresAtNameNode(dm, dns.get(1), true, dn2Vol1.getAbsolutePath());
|
|
|
|
|
|
// Reconfigure a third time with the failed volumes. Afterwards, we expect
|
|
// Reconfigure a third time with the failed volumes. Afterwards, we expect
|
|
// the same volume failures to be reported. (No double-counting.)
|
|
// the same volume failures to be reported. (No double-counting.)
|
|
@@ -427,8 +434,8 @@ public class TestDataNodeVolumeFailureReporting {
|
|
DataNodeTestUtils.triggerHeartbeat(dns.get(0));
|
|
DataNodeTestUtils.triggerHeartbeat(dns.get(0));
|
|
DataNodeTestUtils.triggerHeartbeat(dns.get(1));
|
|
DataNodeTestUtils.triggerHeartbeat(dns.get(1));
|
|
|
|
|
|
- checkFailuresAtDataNode(dns.get(0), 1, false, dn1Vol1.getAbsolutePath());
|
|
|
|
- checkFailuresAtDataNode(dns.get(1), 1, false, dn2Vol1.getAbsolutePath());
|
|
|
|
|
|
+ checkFailuresAtDataNode(dns.get(0), 1, true, dn1Vol1.getAbsolutePath());
|
|
|
|
+ checkFailuresAtDataNode(dns.get(1), 1, true, dn2Vol1.getAbsolutePath());
|
|
|
|
|
|
// Ensure we wait a sufficient amount of time.
|
|
// Ensure we wait a sufficient amount of time.
|
|
assert (WAIT_FOR_HEARTBEATS * 10) > WAIT_FOR_DEATH;
|
|
assert (WAIT_FOR_HEARTBEATS * 10) > WAIT_FOR_DEATH;
|
|
@@ -436,9 +443,9 @@ public class TestDataNodeVolumeFailureReporting {
|
|
// The NN reports two volume failures again.
|
|
// The NN reports two volume failures again.
|
|
DFSTestUtil.waitForDatanodeStatus(dm, 3, 0, 2,
|
|
DFSTestUtil.waitForDatanodeStatus(dm, 3, 0, 2,
|
|
origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
|
|
origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
|
|
- checkAggregateFailuresAtNameNode(false, 2);
|
|
|
|
- checkFailuresAtNameNode(dm, dns.get(0), false, dn1Vol1.getAbsolutePath());
|
|
|
|
- checkFailuresAtNameNode(dm, dns.get(1), false, dn2Vol1.getAbsolutePath());
|
|
|
|
|
|
+ checkAggregateFailuresAtNameNode(true, 2);
|
|
|
|
+ checkFailuresAtNameNode(dm, dns.get(0), true, dn1Vol1.getAbsolutePath());
|
|
|
|
+ checkFailuresAtNameNode(dm, dns.get(1), true, dn2Vol1.getAbsolutePath());
|
|
|
|
|
|
// Replace failed volume with healthy volume and run reconfigure DataNode.
|
|
// Replace failed volume with healthy volume and run reconfigure DataNode.
|
|
// The failed volume information should be cleared.
|
|
// The failed volume information should be cleared.
|
|
@@ -514,6 +521,95 @@ public class TestDataNodeVolumeFailureReporting {
|
|
currentVersion.exists());
|
|
currentVersion.exists());
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Verify DataNode NumFailedVolumes and FailedStorageLocations
|
|
|
|
+ * after hot swap out of failed volume.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testHotSwapOutFailedVolumeAndReporting()
|
|
|
|
+ throws Exception {
|
|
|
|
+ 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);
|
|
|
|
+
|
|
|
|
+ MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
|
|
|
|
+ ObjectName mxbeanName = new ObjectName(
|
|
|
|
+ "Hadoop:service=DataNode,name=FSDatasetState-" + dn0.getDatanodeUuid());
|
|
|
|
+ int numFailedVolumes = (int) mbs.getAttribute(mxbeanName,
|
|
|
|
+ "NumFailedVolumes");
|
|
|
|
+ Assert.assertEquals(dn0.getFSDataset().getNumFailedVolumes(),
|
|
|
|
+ numFailedVolumes);
|
|
|
|
+ checkFailuresAtDataNode(dn0, 0, false, new String[] {});
|
|
|
|
+
|
|
|
|
+ // Fail dn0Vol1 first.
|
|
|
|
+ // Verify NumFailedVolumes and FailedStorageLocations are empty.
|
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
|
|
|
|
+ DataNodeTestUtils.waitForDiskError(dn0,
|
|
|
|
+ DataNodeTestUtils.getVolume(dn0, dn0Vol1));
|
|
|
|
+ numFailedVolumes = (int) mbs.getAttribute(mxbeanName, "NumFailedVolumes");
|
|
|
|
+ Assert.assertEquals(1, numFailedVolumes);
|
|
|
|
+ Assert.assertEquals(dn0.getFSDataset().getNumFailedVolumes(),
|
|
|
|
+ numFailedVolumes);
|
|
|
|
+ checkFailuresAtDataNode(dn0, 1, true,
|
|
|
|
+ new String[] {dn0Vol1.getAbsolutePath()});
|
|
|
|
+
|
|
|
|
+ // Reconfigure disks without fixing the failed disk.
|
|
|
|
+ // Verify NumFailedVolumes and FailedStorageLocations haven't changed.
|
|
|
|
+ try {
|
|
|
|
+ dn0.reconfigurePropertyImpl(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
|
|
|
|
+ oldDataDirs);
|
|
|
|
+ fail("Reconfigure with failed disk should throw exception.");
|
|
|
|
+ } catch (ReconfigurationException e) {
|
|
|
|
+ Assert.assertTrue("Reconfigure exception doesn't have expected path!",
|
|
|
|
+ e.getCause().getMessage().contains(dn0Vol1.getAbsolutePath()));
|
|
|
|
+ }
|
|
|
|
+ numFailedVolumes = (int) mbs.getAttribute(mxbeanName, "NumFailedVolumes");
|
|
|
|
+ Assert.assertEquals(1, numFailedVolumes);
|
|
|
|
+ Assert.assertEquals(dn0.getFSDataset().getNumFailedVolumes(),
|
|
|
|
+ numFailedVolumes);
|
|
|
|
+ checkFailuresAtDataNode(dn0, 1, true,
|
|
|
|
+ new String[] {dn0Vol1.getAbsolutePath()});
|
|
|
|
+
|
|
|
|
+ // Hot swap out the failed volume.
|
|
|
|
+ // Verify NumFailedVolumes and FailedStorageLocations are reset.
|
|
|
|
+ String dataDirs = dn0Vol2.getPath();
|
|
|
|
+ dn0.reconfigurePropertyImpl(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
|
|
|
|
+ dataDirs);
|
|
|
|
+ numFailedVolumes = (int) mbs.getAttribute(mxbeanName, "NumFailedVolumes");
|
|
|
|
+ Assert.assertEquals(0, numFailedVolumes);
|
|
|
|
+ Assert.assertEquals(dn0.getFSDataset().getNumFailedVolumes(),
|
|
|
|
+ numFailedVolumes);
|
|
|
|
+ checkFailuresAtDataNode(dn0, 0, true, new String[] {});
|
|
|
|
+
|
|
|
|
+ // Fix failure volume dn0Vol1 and remount it back.
|
|
|
|
+ // Verify NumFailedVolumes and FailedStorageLocations are empty.
|
|
|
|
+ DataNodeTestUtils.restoreDataDirFromFailure(dn0Vol1);
|
|
|
|
+ dn0.reconfigurePropertyImpl(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
|
|
|
|
+ oldDataDirs);
|
|
|
|
+ numFailedVolumes = (int) mbs.getAttribute(mxbeanName, "NumFailedVolumes");
|
|
|
|
+ Assert.assertEquals(0, numFailedVolumes);
|
|
|
|
+ Assert.assertEquals(dn0.getFSDataset().getNumFailedVolumes(),
|
|
|
|
+ numFailedVolumes);
|
|
|
|
+ checkFailuresAtDataNode(dn0, 0, true, new String[] {});
|
|
|
|
+
|
|
|
|
+ // Fail dn0Vol2.
|
|
|
|
+ // Verify NumFailedVolumes and FailedStorageLocations are updated.
|
|
|
|
+ DataNodeTestUtils.injectDataDirFailure(dn0Vol2);
|
|
|
|
+ DataNodeTestUtils.waitForDiskError(dn0,
|
|
|
|
+ DataNodeTestUtils.getVolume(dn0, dn0Vol2));
|
|
|
|
+ numFailedVolumes = (int) mbs.getAttribute(mxbeanName, "NumFailedVolumes");
|
|
|
|
+ Assert.assertEquals(1, numFailedVolumes);
|
|
|
|
+ Assert.assertEquals(dn0.getFSDataset().getNumFailedVolumes(),
|
|
|
|
+ numFailedVolumes);
|
|
|
|
+ checkFailuresAtDataNode(dn0, 1, true,
|
|
|
|
+ new String[] {dn0Vol2.getAbsolutePath()});
|
|
|
|
+
|
|
|
|
+ // Verify DataNode tolerating one disk failure.
|
|
|
|
+ assertTrue(dn0.shouldRun());
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Checks the NameNode for correct values of aggregate counters tracking failed
|
|
* Checks the NameNode for correct values of aggregate counters tracking failed
|
|
* volumes across all DataNodes.
|
|
* volumes across all DataNodes.
|