|
@@ -25,6 +25,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHEC
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIFELINE_RPC_ADDRESS_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY;
|
|
|
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerFaultInjector;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
|
|
|
import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
|
|
|
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
|
@@ -95,6 +98,12 @@ public class TestDataNodeLifeline {
|
|
|
private FSNamesystem namesystem;
|
|
|
private DataNode dn;
|
|
|
private BPServiceActor bpsa;
|
|
|
+ private final BlockManagerFaultInjector injector = new BlockManagerFaultInjector() {
|
|
|
+ @Override
|
|
|
+ public void mockAnException() {
|
|
|
+ throw new UnknownError("Unknown exception");
|
|
|
+ }
|
|
|
+ };
|
|
|
|
|
|
@Before
|
|
|
public void setup() throws Exception {
|
|
@@ -336,4 +345,49 @@ public class TestDataNodeLifeline {
|
|
|
return result;
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Mock an exception in HeartbeatManager#updateHeartbeat and HeartbeatManager#updateLifeline
|
|
|
+ * respectively, and trigger the heartbeat and lifeline in sequence. The capacityTotal obtained
|
|
|
+ * before and after this operation should be the same.
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testHeartbeatAndLifelineOnError() throws Exception {
|
|
|
+ final Configuration config = new HdfsConfiguration();
|
|
|
+ config.set(DFS_NAMENODE_LIFELINE_RPC_ADDRESS_KEY, "0.0.0.0:0");
|
|
|
+
|
|
|
+ try(MiniDFSCluster cluster =
|
|
|
+ new MiniDFSCluster.Builder(config).numDataNodes(1).build()) {
|
|
|
+ cluster.waitActive();
|
|
|
+ final FSNamesystem fsNamesystem = cluster.getNamesystem();
|
|
|
+
|
|
|
+ // Get capacityTotal before triggering heartbeat and lifeline.
|
|
|
+ DatanodeStatistics datanodeStatistics =
|
|
|
+ fsNamesystem.getBlockManager().getDatanodeManager().getDatanodeStatistics();
|
|
|
+ long capacityTotalBefore = datanodeStatistics.getCapacityTotal();
|
|
|
+
|
|
|
+ // Mock an exception in HeartbeatManager#updateHeartbeat and HeartbeatManager#updateLifeline.
|
|
|
+ BlockManagerFaultInjector.instance = injector;
|
|
|
+ DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
+ BlockPoolManager blockPoolManager = dataNode.getBlockPoolManager();
|
|
|
+ for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
|
|
|
+ if (bpos != null) {
|
|
|
+ for (BPServiceActor actor : bpos.getBPServiceActors()) {
|
|
|
+ try {
|
|
|
+ actor.triggerHeartbeatForTests();
|
|
|
+ actor.sendLifelineForTests();
|
|
|
+ } catch (Throwable e) {
|
|
|
+ assertTrue(e.getMessage().contains("Unknown exception"));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Get capacityTotal after triggering heartbeat and lifeline.
|
|
|
+ long capacityTotalAfter = datanodeStatistics.getCapacityTotal();
|
|
|
+ // The capacityTotal should be same.
|
|
|
+ assertEquals(capacityTotalBefore, capacityTotalAfter);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|