|
@@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
|
|
@@ -87,6 +88,9 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
private final static File TEMP_DIR = new File(System.getProperty(
|
|
|
"test.build.data", "/tmp"), "decommision");
|
|
|
private final File hostFile = new File(TEMP_DIR + File.separator + "hostFile.txt");
|
|
|
+ private final File excludeHostFile = new File(TEMP_DIR + File.separator +
|
|
|
+ "excludeHostFile.txt");
|
|
|
+
|
|
|
private MockRM rm;
|
|
|
|
|
|
/**
|
|
@@ -221,6 +225,109 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
checkDecommissionedNMCount(rm, metricCount + 1);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Graceful decommission node with no running application.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testGracefulDecommissionNoApp() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH, hostFile
|
|
|
+ .getAbsolutePath());
|
|
|
+
|
|
|
+ writeToHostsFile("");
|
|
|
+ rm = new MockRM(conf);
|
|
|
+ rm.start();
|
|
|
+
|
|
|
+ MockNM nm1 = rm.registerNode("host1:1234", 5120);
|
|
|
+ MockNM nm2 = rm.registerNode("host2:5678", 10240);
|
|
|
+ MockNM nm3 = rm.registerNode("host3:4433", 5120);
|
|
|
+
|
|
|
+ int metricCount = ClusterMetrics.getMetrics().getNumDecommisionedNMs();
|
|
|
+ NodeHeartbeatResponse nodeHeartbeat1 = nm1.nodeHeartbeat(true);
|
|
|
+ NodeHeartbeatResponse nodeHeartbeat2 = nm2.nodeHeartbeat(true);
|
|
|
+ NodeHeartbeatResponse nodeHeartbeat3 = nm3.nodeHeartbeat(true);
|
|
|
+
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat1.getNodeAction()));
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat2.getNodeAction()));
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat3.getNodeAction()));
|
|
|
+
|
|
|
+ rm.waitForState(nm2.getNodeId(), NodeState.RUNNING);
|
|
|
+ rm.waitForState(nm3.getNodeId(), NodeState.RUNNING);
|
|
|
+
|
|
|
+ // Graceful decommission both host2 and host3.
|
|
|
+ writeToHostsFile("host2", "host3");
|
|
|
+ rm.getNodesListManager().refreshNodes(conf, true);
|
|
|
+
|
|
|
+ rm.waitForState(nm2.getNodeId(), NodeState.DECOMMISSIONING);
|
|
|
+ rm.waitForState(nm3.getNodeId(), NodeState.DECOMMISSIONING);
|
|
|
+
|
|
|
+ nodeHeartbeat1 = nm1.nodeHeartbeat(true);
|
|
|
+ nodeHeartbeat2 = nm2.nodeHeartbeat(true);
|
|
|
+ nodeHeartbeat3 = nm3.nodeHeartbeat(true);
|
|
|
+
|
|
|
+ checkDecommissionedNMCount(rm, metricCount + 2);
|
|
|
+ rm.waitForState(nm2.getNodeId(), NodeState.DECOMMISSIONED);
|
|
|
+ rm.waitForState(nm3.getNodeId(), NodeState.DECOMMISSIONED);
|
|
|
+
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat1.getNodeAction()));
|
|
|
+ Assert.assertEquals(NodeAction.SHUTDOWN, nodeHeartbeat2.getNodeAction());
|
|
|
+ Assert.assertEquals(NodeAction.SHUTDOWN, nodeHeartbeat3.getNodeAction());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Graceful decommission node with running application.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testGracefulDecommissionWithApp() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH, hostFile
|
|
|
+ .getAbsolutePath());
|
|
|
+
|
|
|
+ writeToHostsFile("");
|
|
|
+ rm = new MockRM(conf);
|
|
|
+ rm.start();
|
|
|
+
|
|
|
+ MockNM nm1 = rm.registerNode("host1:1234", 10240);
|
|
|
+ MockNM nm2 = rm.registerNode("host2:5678", 20480);
|
|
|
+ MockNM nm3 = rm.registerNode("host3:4433", 10240);
|
|
|
+ NodeId id1 = nm1.getNodeId();
|
|
|
+ NodeId id3 = nm3.getNodeId();
|
|
|
+ rm.waitForState(id1, NodeState.RUNNING);
|
|
|
+ rm.waitForState(id3, NodeState.RUNNING);
|
|
|
+
|
|
|
+ // Create an app and launch two containers on host1.
|
|
|
+ RMApp app = rm.submitApp(2000);
|
|
|
+ MockAM am = MockRM.launchAndRegisterAM(app, rm, nm1);
|
|
|
+ ApplicationAttemptId aaid = app.getCurrentAppAttempt().getAppAttemptId();
|
|
|
+
|
|
|
+ // Graceful decommission host1 and host3
|
|
|
+ writeToHostsFile("host1", "host3");
|
|
|
+ rm.getNodesListManager().refreshNodes(conf, true);
|
|
|
+ rm.waitForState(id1, NodeState.DECOMMISSIONING);
|
|
|
+ rm.waitForState(id3, NodeState.DECOMMISSIONING);
|
|
|
+
|
|
|
+ // host1 should be DECOMMISSIONING due to running containers.
|
|
|
+ // host3 should become DECOMMISSIONED.
|
|
|
+ nm1.nodeHeartbeat(aaid, 2, ContainerState.RUNNING);
|
|
|
+ nm3.nodeHeartbeat(true);
|
|
|
+ rm.waitForState(id1, NodeState.DECOMMISSIONING);
|
|
|
+ rm.waitForState(id3, NodeState.DECOMMISSIONED);
|
|
|
+ nm1.nodeHeartbeat(aaid, 2, ContainerState.RUNNING);
|
|
|
+
|
|
|
+ // Complete containers on host1.
|
|
|
+ // Since the app is still RUNNING, expect NodeAction.NORMAL.
|
|
|
+ NodeHeartbeatResponse nodeHeartbeat1 =
|
|
|
+ nm1.nodeHeartbeat(aaid, 2, ContainerState.COMPLETE);
|
|
|
+ Assert.assertEquals(NodeAction.NORMAL, nodeHeartbeat1.getNodeAction());
|
|
|
+
|
|
|
+ // Finish the app and verified DECOMMISSIONED.
|
|
|
+ MockRM.finishAMAndVerifyAppState(app, rm, nm1, am);
|
|
|
+ rm.waitForState(app.getApplicationId(), RMAppState.FINISHED);
|
|
|
+ nodeHeartbeat1 = nm1.nodeHeartbeat(aaid, 2, ContainerState.COMPLETE);
|
|
|
+ Assert.assertEquals(NodeAction.SHUTDOWN, nodeHeartbeat1.getNodeAction());
|
|
|
+ rm.waitForState(id1, NodeState.DECOMMISSIONED);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Decommissioning using a post-configured include hosts file
|
|
|
*/
|
|
@@ -1225,19 +1332,17 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
MockNM nm1 = new MockNM("host1:1234", 5120, resourceTrackerService);
|
|
|
RegisterNodeManagerResponse response = nm1.registerNode();
|
|
|
Assert.assertEquals(NodeAction.NORMAL, response.getNodeAction());
|
|
|
+ int shutdownNMsCount = ClusterMetrics.getMetrics().getNumShutdownNMs();
|
|
|
writeToHostsFile("host2");
|
|
|
conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH,
|
|
|
hostFile.getAbsolutePath());
|
|
|
rm.getNodesListManager().refreshNodes(conf);
|
|
|
NodeHeartbeatResponse heartbeatResponse = nm1.nodeHeartbeat(true);
|
|
|
Assert.assertEquals(NodeAction.SHUTDOWN, heartbeatResponse.getNodeAction());
|
|
|
- int shutdownNMsCount = ClusterMetrics.getMetrics().getNumShutdownNMs();
|
|
|
- checkShutdownNMCount(rm, shutdownNMsCount);
|
|
|
checkDecommissionedNMCount(rm, decommisionedNMsCount);
|
|
|
request.setNodeId(nm1.getNodeId());
|
|
|
resourceTrackerService.unRegisterNodeManager(request);
|
|
|
- shutdownNMsCount = ClusterMetrics.getMetrics().getNumShutdownNMs();
|
|
|
- checkShutdownNMCount(rm, shutdownNMsCount);
|
|
|
+ checkShutdownNMCount(rm, ++shutdownNMsCount);
|
|
|
checkDecommissionedNMCount(rm, decommisionedNMsCount);
|
|
|
|
|
|
// 1. Register the Node Manager
|
|
@@ -1273,8 +1378,6 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
nm1.nodeHeartbeat(true);
|
|
|
nm2.nodeHeartbeat(true);
|
|
|
|
|
|
- File excludeHostFile =
|
|
|
- new File(TEMP_DIR + File.separator + "excludeHostFile.txt");
|
|
|
writeToHostsFile(excludeHostFile, "host1");
|
|
|
conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
|
|
|
excludeHostFile.getAbsolutePath());
|
|
@@ -1300,8 +1403,6 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
Assert.assertEquals("The inactiveRMNodes should contain an entry for the" +
|
|
|
"decommissioned node",
|
|
|
1, rm1.getRMContext().getInactiveRMNodes().size());
|
|
|
- excludeHostFile =
|
|
|
- new File(TEMP_DIR + File.separator + "excludeHostFile.txt");
|
|
|
writeToHostsFile(excludeHostFile, "");
|
|
|
conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
|
|
|
excludeHostFile.getAbsolutePath());
|
|
@@ -1331,8 +1432,6 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
nm1.nodeHeartbeat(true);
|
|
|
nm2.nodeHeartbeat(true);
|
|
|
//host3 will not register or heartbeat
|
|
|
- File excludeHostFile =
|
|
|
- new File(TEMP_DIR + File.separator + "excludeHostFile.txt");
|
|
|
writeToHostsFile(excludeHostFile, "host3", "host2");
|
|
|
conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
|
|
|
excludeHostFile.getAbsolutePath());
|
|
@@ -1364,14 +1463,12 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
MockNM nm2 = rm.registerNode("host2:5678", 10240);
|
|
|
nm1.nodeHeartbeat(true);
|
|
|
nm2.nodeHeartbeat(true);
|
|
|
- File excludeHostFile =
|
|
|
- new File(TEMP_DIR + File.separator + "excludeHostFile.txt");
|
|
|
writeToHostsFile(excludeHostFile, "host3", "host2");
|
|
|
conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
|
|
|
excludeHostFile.getAbsolutePath());
|
|
|
writeToHostsFile(hostFile, "host1", "host2");
|
|
|
writeToHostsFile(excludeHostFile, "host1");
|
|
|
- rm.getNodesListManager().refreshNodesGracefully(conf);
|
|
|
+ rm.getNodesListManager().refreshNodesGracefully(conf, null);
|
|
|
rm.drainEvents();
|
|
|
nm1.nodeHeartbeat(true);
|
|
|
rm.drainEvents();
|
|
@@ -1380,7 +1477,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
.getInactiveRMNodes().get(nm1.getNodeId()).getState() == NodeState
|
|
|
.DECOMMISSIONED);
|
|
|
writeToHostsFile(excludeHostFile, "");
|
|
|
- rm.getNodesListManager().refreshNodesGracefully(conf);
|
|
|
+ rm.getNodesListManager().refreshNodesGracefully(conf, null);
|
|
|
rm.drainEvents();
|
|
|
Assert.assertTrue("Node " + nm1.getNodeId().getHost() +
|
|
|
" should be Decommissioned", rm.getRMContext()
|
|
@@ -1390,7 +1487,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Remove a node from all lists and check if its forgotten
|
|
|
+ * Remove a node from all lists and check if its forgotten.
|
|
|
*/
|
|
|
@Test
|
|
|
public void testNodeRemovalNormally() throws Exception {
|
|
@@ -1411,7 +1508,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
public void refreshNodesOption(boolean doGraceful, Configuration conf)
|
|
|
throws Exception {
|
|
|
if (doGraceful) {
|
|
|
- rm.getNodesListManager().refreshNodesGracefully(conf);
|
|
|
+ rm.getNodesListManager().refreshNodesGracefully(conf, null);
|
|
|
} else {
|
|
|
rm.getNodesListManager().refreshNodes(conf);
|
|
|
}
|
|
@@ -1420,8 +1517,6 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
public void testNodeRemovalUtil(boolean doGraceful) throws Exception {
|
|
|
Configuration conf = new Configuration();
|
|
|
int timeoutValue = 500;
|
|
|
- File excludeHostFile = new File(TEMP_DIR + File.separator +
|
|
|
- "excludeHostFile.txt");
|
|
|
conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH, "");
|
|
|
conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH, "");
|
|
|
conf.setInt(YarnConfiguration.RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC,
|
|
@@ -1455,18 +1550,23 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH, hostFile
|
|
|
.getAbsolutePath());
|
|
|
refreshNodesOption(doGraceful, conf);
|
|
|
+ if (doGraceful) {
|
|
|
+ rm.waitForState(nm2.getNodeId(), NodeState.DECOMMISSIONING);
|
|
|
+ }
|
|
|
nm1.nodeHeartbeat(true);
|
|
|
+ nm2.nodeHeartbeat(true);
|
|
|
rm.drainEvents();
|
|
|
Assert.assertTrue("Node should not be in active node list",
|
|
|
!rmContext.getRMNodes().containsKey(nm2.getNodeId()));
|
|
|
|
|
|
RMNode rmNode = rmContext.getInactiveRMNodes().get(nm2.getNodeId());
|
|
|
Assert.assertEquals("Node should be in inactive node list",
|
|
|
- rmNode.getState(), NodeState.SHUTDOWN);
|
|
|
+ rmNode.getState(),
|
|
|
+ doGraceful? NodeState.DECOMMISSIONED : NodeState.SHUTDOWN);
|
|
|
Assert.assertEquals("Active nodes should be 2",
|
|
|
metrics.getNumActiveNMs(), 2);
|
|
|
- Assert.assertEquals("Shutdown nodes should be 1",
|
|
|
- metrics.getNumShutdownNMs(), 1);
|
|
|
+ Assert.assertEquals("Shutdown nodes should be expected",
|
|
|
+ metrics.getNumShutdownNMs(), doGraceful? 0 : 1);
|
|
|
|
|
|
int nodeRemovalTimeout =
|
|
|
conf.getInt(
|
|
@@ -1491,14 +1591,18 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
rm.drainEvents();
|
|
|
writeToHostsFile("host1", ip);
|
|
|
refreshNodesOption(doGraceful, conf);
|
|
|
+ rm.waitForState(nm2.getNodeId(),
|
|
|
+ doGraceful? NodeState.DECOMMISSIONING : NodeState.SHUTDOWN);
|
|
|
+ nm2.nodeHeartbeat(true);
|
|
|
rm.drainEvents();
|
|
|
rmNode = rmContext.getInactiveRMNodes().get(nm2.getNodeId());
|
|
|
Assert.assertEquals("Node should be shutdown",
|
|
|
- rmNode.getState(), NodeState.SHUTDOWN);
|
|
|
+ rmNode.getState(),
|
|
|
+ doGraceful? NodeState.DECOMMISSIONED : NodeState.SHUTDOWN);
|
|
|
Assert.assertEquals("Active nodes should be 2",
|
|
|
metrics.getNumActiveNMs(), 2);
|
|
|
- Assert.assertEquals("Shutdown nodes should be 1",
|
|
|
- metrics.getNumShutdownNMs(), 1);
|
|
|
+ Assert.assertEquals("Shutdown nodes should be expected",
|
|
|
+ metrics.getNumShutdownNMs(), doGraceful? 0 : 1);
|
|
|
|
|
|
//add back the node before timer expires
|
|
|
latch.await(maxThreadSleeptime - 2000, TimeUnit.MILLISECONDS);
|
|
@@ -1542,6 +1646,20 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
}
|
|
|
|
|
|
//Test decommed/ing node that transitions to untracked,timer should remove
|
|
|
+ testNodeRemovalUtilDecomToUntracked(rmContext, conf, nm1, nm2, nm3,
|
|
|
+ maxThreadSleeptime, doGraceful);
|
|
|
+ rm.stop();
|
|
|
+ }
|
|
|
+
|
|
|
+ // A helper method used by testNodeRemovalUtil to avoid exceeding
|
|
|
+ // max allowed length.
|
|
|
+ private void testNodeRemovalUtilDecomToUntracked(
|
|
|
+ RMContext rmContext, Configuration conf,
|
|
|
+ MockNM nm1, MockNM nm2, MockNM nm3,
|
|
|
+ long maxThreadSleeptime, boolean doGraceful) throws Exception {
|
|
|
+ ClusterMetrics metrics = ClusterMetrics.getMetrics();
|
|
|
+ String ip = NetUtils.normalizeHostName("localhost");
|
|
|
+ CountDownLatch latch = new CountDownLatch(1);
|
|
|
writeToHostsFile("host1", ip, "host2");
|
|
|
writeToHostsFile(excludeHostFile, "host2");
|
|
|
refreshNodesOption(doGraceful, conf);
|
|
@@ -1549,7 +1667,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
//nm2.nodeHeartbeat(true);
|
|
|
nm3.nodeHeartbeat(true);
|
|
|
latch.await(maxThreadSleeptime, TimeUnit.MILLISECONDS);
|
|
|
- rmNode = doGraceful ? rmContext.getRMNodes().get(nm2.getNodeId()) :
|
|
|
+ RMNode rmNode = doGraceful ? rmContext.getRMNodes().get(nm2.getNodeId()) :
|
|
|
rmContext.getInactiveRMNodes().get(nm2.getNodeId());
|
|
|
Assert.assertNotEquals("Timer for this node was not canceled!",
|
|
|
rmNode, null);
|
|
@@ -1560,6 +1678,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
writeToHostsFile("host1", ip);
|
|
|
writeToHostsFile(excludeHostFile, "");
|
|
|
refreshNodesOption(doGraceful, conf);
|
|
|
+ nm2.nodeHeartbeat(true);
|
|
|
latch.await(maxThreadSleeptime, TimeUnit.MILLISECONDS);
|
|
|
rmNode = doGraceful ? rmContext.getRMNodes().get(nm2.getNodeId()) :
|
|
|
rmContext.getInactiveRMNodes().get(nm2.getNodeId());
|
|
@@ -1571,16 +1690,12 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
metrics.getNumShutdownNMs(), 0);
|
|
|
Assert.assertEquals("Active nodes should be 2",
|
|
|
metrics.getNumActiveNMs(), 2);
|
|
|
-
|
|
|
- rm.stop();
|
|
|
}
|
|
|
|
|
|
private void testNodeRemovalUtilLost(boolean doGraceful) throws Exception {
|
|
|
Configuration conf = new Configuration();
|
|
|
conf.setLong(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS, 2000);
|
|
|
int timeoutValue = 500;
|
|
|
- File excludeHostFile = new File(TEMP_DIR + File.separator +
|
|
|
- "excludeHostFile.txt");
|
|
|
conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH,
|
|
|
hostFile.getAbsolutePath());
|
|
|
conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
|
|
@@ -1613,7 +1728,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
Assert.assertEquals("All 3 nodes should be active",
|
|
|
metrics.getNumActiveNMs(), 3);
|
|
|
int waitCount = 0;
|
|
|
- while(waitCount ++<20){
|
|
|
+ while(waitCount++ < 20){
|
|
|
synchronized (this) {
|
|
|
wait(200);
|
|
|
}
|
|
@@ -1665,8 +1780,6 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
throws Exception {
|
|
|
Configuration conf = new Configuration();
|
|
|
int timeoutValue = 500;
|
|
|
- File excludeHostFile = new File(TEMP_DIR + File.separator +
|
|
|
- "excludeHostFile.txt");
|
|
|
conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH,
|
|
|
hostFile.getAbsolutePath());
|
|
|
conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
|
|
@@ -1737,8 +1850,6 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
throws Exception {
|
|
|
Configuration conf = new Configuration();
|
|
|
int timeoutValue = 500;
|
|
|
- File excludeHostFile = new File(TEMP_DIR + File.separator +
|
|
|
- "excludeHostFile.txt");
|
|
|
conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH,
|
|
|
hostFile.getAbsolutePath());
|
|
|
conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
|
|
@@ -1782,15 +1893,19 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
nm2.nodeHeartbeat(false);
|
|
|
nm3.nodeHeartbeat(true);
|
|
|
rm.drainEvents();
|
|
|
- Assert.assertNotEquals("host2 should be a shutdown NM!",
|
|
|
- rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null);
|
|
|
- Assert.assertEquals("host2 should be a shutdown NM!",
|
|
|
- rmContext.getInactiveRMNodes().get(nm2.getNodeId()).getState(),
|
|
|
- NodeState.SHUTDOWN);
|
|
|
+ if (!doGraceful) {
|
|
|
+ Assert.assertNotEquals("host2 should be a shutdown NM!",
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null);
|
|
|
+ Assert.assertEquals("host2 should be a shutdown NM!",
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId()).getState(),
|
|
|
+ NodeState.SHUTDOWN);
|
|
|
+ }
|
|
|
Assert.assertEquals("There should be 2 Active NM!",
|
|
|
clusterMetrics.getNumActiveNMs(), 2);
|
|
|
- Assert.assertEquals("There should be 1 Shutdown NM!",
|
|
|
- clusterMetrics.getNumShutdownNMs(), 1);
|
|
|
+ if (!doGraceful) {
|
|
|
+ Assert.assertEquals("There should be 1 Shutdown NM!",
|
|
|
+ clusterMetrics.getNumShutdownNMs(), 1);
|
|
|
+ }
|
|
|
Assert.assertEquals("There should be 0 Unhealthy NM!",
|
|
|
clusterMetrics.getUnhealthyNMs(), 0);
|
|
|
int nodeRemovalTimeout =
|
|
@@ -1818,7 +1933,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
}
|
|
|
|
|
|
private void writeToHostsFile(String... hosts) throws IOException {
|
|
|
- writeToHostsFile(hostFile, hosts);
|
|
|
+ writeToHostsFile(hostFile, hosts);
|
|
|
}
|
|
|
|
|
|
private void writeToHostsFile(File file, String... hosts)
|