|
@@ -31,6 +31,8 @@ import java.util.Collections;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
|
|
+import java.util.concurrent.CountDownLatch;
|
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
@@ -48,8 +50,6 @@ import org.apache.hadoop.yarn.api.records.NodeState;
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
-import org.apache.hadoop.yarn.event.Dispatcher;
|
|
|
|
-import org.apache.hadoop.yarn.event.DrainDispatcher;
|
|
|
|
import org.apache.hadoop.yarn.event.Event;
|
|
import org.apache.hadoop.yarn.event.Event;
|
|
import org.apache.hadoop.yarn.event.EventDispatcher;
|
|
import org.apache.hadoop.yarn.event.EventDispatcher;
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
@@ -142,12 +142,12 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
|
|
|
rm.getNodesListManager().refreshNodes(conf);
|
|
rm.getNodesListManager().refreshNodes(conf);
|
|
|
|
|
|
- checkDecommissionedNMCount(rm, ++metricCount);
|
|
|
|
|
|
+ checkShutdownNMCount(rm, ++metricCount);
|
|
|
|
|
|
nodeHeartbeat = nm1.nodeHeartbeat(true);
|
|
nodeHeartbeat = nm1.nodeHeartbeat(true);
|
|
Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
Assert
|
|
Assert
|
|
- .assertEquals(1, ClusterMetrics.getMetrics().getNumDecommisionedNMs());
|
|
|
|
|
|
+ .assertEquals(1, ClusterMetrics.getMetrics().getNumShutdownNMs());
|
|
|
|
|
|
nodeHeartbeat = nm2.nodeHeartbeat(true);
|
|
nodeHeartbeat = nm2.nodeHeartbeat(true);
|
|
Assert.assertTrue("Node is not decommisioned.", NodeAction.SHUTDOWN
|
|
Assert.assertTrue("Node is not decommisioned.", NodeAction.SHUTDOWN
|
|
@@ -156,7 +156,8 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
nodeHeartbeat = nm3.nodeHeartbeat(true);
|
|
nodeHeartbeat = nm3.nodeHeartbeat(true);
|
|
Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
Assert.assertEquals(metricCount, ClusterMetrics.getMetrics()
|
|
Assert.assertEquals(metricCount, ClusterMetrics.getMetrics()
|
|
- .getNumDecommisionedNMs());
|
|
|
|
|
|
+ .getNumShutdownNMs());
|
|
|
|
+ rm.stop();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -227,7 +228,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
MockNM nm2 = rm.registerNode("host2:5678", 10240);
|
|
MockNM nm2 = rm.registerNode("host2:5678", 10240);
|
|
ClusterMetrics metrics = ClusterMetrics.getMetrics();
|
|
ClusterMetrics metrics = ClusterMetrics.getMetrics();
|
|
assert(metrics != null);
|
|
assert(metrics != null);
|
|
- int initialMetricCount = metrics.getNumDecommisionedNMs();
|
|
|
|
|
|
+ int initialMetricCount = metrics.getNumShutdownNMs();
|
|
NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true);
|
|
NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true);
|
|
Assert.assertEquals(
|
|
Assert.assertEquals(
|
|
NodeAction.NORMAL,
|
|
NodeAction.NORMAL,
|
|
@@ -240,16 +241,16 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH, hostFile
|
|
conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH, hostFile
|
|
.getAbsolutePath());
|
|
.getAbsolutePath());
|
|
rm.getNodesListManager().refreshNodes(conf);
|
|
rm.getNodesListManager().refreshNodes(conf);
|
|
- checkDecommissionedNMCount(rm, ++initialMetricCount);
|
|
|
|
|
|
+ checkShutdownNMCount(rm, ++initialMetricCount);
|
|
nodeHeartbeat = nm1.nodeHeartbeat(true);
|
|
nodeHeartbeat = nm1.nodeHeartbeat(true);
|
|
Assert.assertEquals(
|
|
Assert.assertEquals(
|
|
- "Node should not have been decomissioned.",
|
|
|
|
|
|
+ "Node should not have been shutdown.",
|
|
NodeAction.NORMAL,
|
|
NodeAction.NORMAL,
|
|
nodeHeartbeat.getNodeAction());
|
|
nodeHeartbeat.getNodeAction());
|
|
- nodeHeartbeat = nm2.nodeHeartbeat(true);
|
|
|
|
- Assert.assertEquals("Node should have been decomissioned but is in state" +
|
|
|
|
- nodeHeartbeat.getNodeAction(),
|
|
|
|
- NodeAction.SHUTDOWN, nodeHeartbeat.getNodeAction());
|
|
|
|
|
|
+ NodeState nodeState =
|
|
|
|
+ rm.getRMContext().getInactiveRMNodes().get(nm2.getNodeId()).getState();
|
|
|
|
+ Assert.assertEquals("Node should have been shutdown but is in state" +
|
|
|
|
+ nodeState, NodeState.SHUTDOWN, nodeState);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -510,7 +511,8 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
Assert.assertNull(nodeLabelsMgr.getNodeLabels().get(nodeId));
|
|
Assert.assertNull(nodeLabelsMgr.getNodeLabels().get(nodeId));
|
|
Assert
|
|
Assert
|
|
.assertFalse(
|
|
.assertFalse(
|
|
- "Node Labels should not accepted by RM If its configured with Central configuration",
|
|
|
|
|
|
+ "Node Labels should not accepted by RM If its configured with " +
|
|
|
|
+ "Central configuration",
|
|
response.getAreNodeLabelsAcceptedByRM());
|
|
response.getAreNodeLabelsAcceptedByRM());
|
|
if (rm != null) {
|
|
if (rm != null) {
|
|
rm.stop();
|
|
rm.stop();
|
|
@@ -892,15 +894,15 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
|
|
|
// node unhealthy
|
|
// node unhealthy
|
|
nm1.nodeHeartbeat(false);
|
|
nm1.nodeHeartbeat(false);
|
|
- checkUnealthyNMCount(rm, nm1, true, 1);
|
|
|
|
|
|
+ checkUnhealthyNMCount(rm, nm1, true, 1);
|
|
|
|
|
|
// node healthy again
|
|
// node healthy again
|
|
nm1.nodeHeartbeat(true);
|
|
nm1.nodeHeartbeat(true);
|
|
- checkUnealthyNMCount(rm, nm1, false, 0);
|
|
|
|
|
|
+ checkUnhealthyNMCount(rm, nm1, false, 0);
|
|
}
|
|
}
|
|
|
|
|
|
- private void checkUnealthyNMCount(MockRM rm, MockNM nm1, boolean health,
|
|
|
|
- int count) throws Exception {
|
|
|
|
|
|
+ private void checkUnhealthyNMCount(MockRM rm, MockNM nm1, boolean health,
|
|
|
|
+ int count) throws Exception {
|
|
|
|
|
|
int waitCount = 0;
|
|
int waitCount = 0;
|
|
while((rm.getRMContext().getRMNodes().get(nm1.getNodeId())
|
|
while((rm.getRMContext().getRMNodes().get(nm1.getNodeId())
|
|
@@ -1002,7 +1004,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
nm1.nodeHeartbeat(true);
|
|
nm1.nodeHeartbeat(true);
|
|
nm2.nodeHeartbeat(false);
|
|
nm2.nodeHeartbeat(false);
|
|
rm.drainEvents();
|
|
rm.drainEvents();
|
|
- checkUnealthyNMCount(rm, nm2, true, 1);
|
|
|
|
|
|
+ checkUnhealthyNMCount(rm, nm2, true, 1);
|
|
final int expectedNMs = ClusterMetrics.getMetrics().getNumActiveNMs();
|
|
final int expectedNMs = ClusterMetrics.getMetrics().getNumActiveNMs();
|
|
QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics();
|
|
QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics();
|
|
// TODO Metrics incorrect in case of the FifoScheduler
|
|
// TODO Metrics incorrect in case of the FifoScheduler
|
|
@@ -1014,7 +1016,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
Assert.assertTrue(NodeAction.NORMAL.equals(response.getNodeAction()));
|
|
Assert.assertTrue(NodeAction.NORMAL.equals(response.getNodeAction()));
|
|
rm.drainEvents();
|
|
rm.drainEvents();
|
|
Assert.assertEquals(expectedNMs, ClusterMetrics.getMetrics().getNumActiveNMs());
|
|
Assert.assertEquals(expectedNMs, ClusterMetrics.getMetrics().getNumActiveNMs());
|
|
- checkUnealthyNMCount(rm, nm2, true, 1);
|
|
|
|
|
|
+ checkUnhealthyNMCount(rm, nm2, true, 1);
|
|
|
|
|
|
// reconnect of unhealthy node
|
|
// reconnect of unhealthy node
|
|
nm2 = rm.registerNode("host2:5678", 5120);
|
|
nm2 = rm.registerNode("host2:5678", 5120);
|
|
@@ -1022,7 +1024,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
Assert.assertTrue(NodeAction.NORMAL.equals(response.getNodeAction()));
|
|
Assert.assertTrue(NodeAction.NORMAL.equals(response.getNodeAction()));
|
|
rm.drainEvents();
|
|
rm.drainEvents();
|
|
Assert.assertEquals(expectedNMs, ClusterMetrics.getMetrics().getNumActiveNMs());
|
|
Assert.assertEquals(expectedNMs, ClusterMetrics.getMetrics().getNumActiveNMs());
|
|
- checkUnealthyNMCount(rm, nm2, true, 1);
|
|
|
|
|
|
+ checkUnhealthyNMCount(rm, nm2, true, 1);
|
|
|
|
|
|
// unhealthy node changed back to healthy
|
|
// unhealthy node changed back to healthy
|
|
nm2 = rm.registerNode("host2:5678", 5120);
|
|
nm2 = rm.registerNode("host2:5678", 5120);
|
|
@@ -1104,7 +1106,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
|
|
|
|
// node unhealthy
|
|
// node unhealthy
|
|
nm1.nodeHeartbeat(false);
|
|
nm1.nodeHeartbeat(false);
|
|
- checkUnealthyNMCount(rm, nm1, true, 1);
|
|
|
|
|
|
+ checkUnhealthyNMCount(rm, nm1, true, 1);
|
|
UnRegisterNodeManagerRequest request = Records
|
|
UnRegisterNodeManagerRequest request = Records
|
|
.newRecord(UnRegisterNodeManagerRequest.class);
|
|
.newRecord(UnRegisterNodeManagerRequest.class);
|
|
request.setNodeId(nm1.getNodeId());
|
|
request.setNodeId(nm1.getNodeId());
|
|
@@ -1119,8 +1121,6 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
rm.start();
|
|
rm.start();
|
|
ResourceTrackerService resourceTrackerService = rm
|
|
ResourceTrackerService resourceTrackerService = rm
|
|
.getResourceTrackerService();
|
|
.getResourceTrackerService();
|
|
- int shutdownNMsCount = ClusterMetrics.getMetrics()
|
|
|
|
- .getNumShutdownNMs();
|
|
|
|
int decommisionedNMsCount = ClusterMetrics.getMetrics()
|
|
int decommisionedNMsCount = ClusterMetrics.getMetrics()
|
|
.getNumDecommisionedNMs();
|
|
.getNumDecommisionedNMs();
|
|
|
|
|
|
@@ -1145,10 +1145,12 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
rm.getNodesListManager().refreshNodes(conf);
|
|
rm.getNodesListManager().refreshNodes(conf);
|
|
NodeHeartbeatResponse heartbeatResponse = nm1.nodeHeartbeat(true);
|
|
NodeHeartbeatResponse heartbeatResponse = nm1.nodeHeartbeat(true);
|
|
Assert.assertEquals(NodeAction.SHUTDOWN, heartbeatResponse.getNodeAction());
|
|
Assert.assertEquals(NodeAction.SHUTDOWN, heartbeatResponse.getNodeAction());
|
|
|
|
+ int shutdownNMsCount = ClusterMetrics.getMetrics().getNumShutdownNMs();
|
|
checkShutdownNMCount(rm, shutdownNMsCount);
|
|
checkShutdownNMCount(rm, shutdownNMsCount);
|
|
- checkDecommissionedNMCount(rm, ++decommisionedNMsCount);
|
|
|
|
|
|
+ checkDecommissionedNMCount(rm, decommisionedNMsCount);
|
|
request.setNodeId(nm1.getNodeId());
|
|
request.setNodeId(nm1.getNodeId());
|
|
resourceTrackerService.unRegisterNodeManager(request);
|
|
resourceTrackerService.unRegisterNodeManager(request);
|
|
|
|
+ shutdownNMsCount = ClusterMetrics.getMetrics().getNumShutdownNMs();
|
|
checkShutdownNMCount(rm, shutdownNMsCount);
|
|
checkShutdownNMCount(rm, shutdownNMsCount);
|
|
checkDecommissionedNMCount(rm, decommisionedNMsCount);
|
|
checkDecommissionedNMCount(rm, decommisionedNMsCount);
|
|
|
|
|
|
@@ -1164,8 +1166,9 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
rm.getNodesListManager().refreshNodes(conf);
|
|
rm.getNodesListManager().refreshNodes(conf);
|
|
request.setNodeId(nm2.getNodeId());
|
|
request.setNodeId(nm2.getNodeId());
|
|
resourceTrackerService.unRegisterNodeManager(request);
|
|
resourceTrackerService.unRegisterNodeManager(request);
|
|
- checkShutdownNMCount(rm, shutdownNMsCount);
|
|
|
|
- checkDecommissionedNMCount(rm, ++decommisionedNMsCount);
|
|
|
|
|
|
+ checkShutdownNMCount(rm, ++shutdownNMsCount);
|
|
|
|
+ checkDecommissionedNMCount(rm, decommisionedNMsCount);
|
|
|
|
+ rm.stop();
|
|
}
|
|
}
|
|
|
|
|
|
@Test(timeout = 30000)
|
|
@Test(timeout = 30000)
|
|
@@ -1300,6 +1303,434 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|
rm.stop();
|
|
rm.stop();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Remove a node from all lists and check if its forgotten
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testNodeRemovalNormally() throws Exception {
|
|
|
|
+ testNodeRemovalUtil(false);
|
|
|
|
+ testNodeRemovalUtilLost(false);
|
|
|
|
+ testNodeRemovalUtilRebooted(false);
|
|
|
|
+ testNodeRemovalUtilUnhealthy(false);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testNodeRemovalGracefully() throws Exception {
|
|
|
|
+ testNodeRemovalUtil(true);
|
|
|
|
+ testNodeRemovalUtilLost(true);
|
|
|
|
+ testNodeRemovalUtilRebooted(true);
|
|
|
|
+ testNodeRemovalUtilUnhealthy(true);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void refreshNodesOption(boolean doGraceful, Configuration conf)
|
|
|
|
+ throws Exception {
|
|
|
|
+ if (doGraceful) {
|
|
|
|
+ rm.getNodesListManager().refreshNodesGracefully(conf);
|
|
|
|
+ } else {
|
|
|
|
+ rm.getNodesListManager().refreshNodes(conf);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ 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,
|
|
|
|
+ timeoutValue);
|
|
|
|
+ CountDownLatch latch = new CountDownLatch(1);
|
|
|
|
+ rm = new MockRM(conf);
|
|
|
|
+ rm.init(conf);
|
|
|
|
+ rm.start();
|
|
|
|
+ RMContext rmContext = rm.getRMContext();
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ MockNM nm1 = rm.registerNode("host1:1234", 5120);
|
|
|
|
+ MockNM nm2 = rm.registerNode("host2:5678", 10240);
|
|
|
|
+ MockNM nm3 = rm.registerNode("localhost:4433", 1024);
|
|
|
|
+ ClusterMetrics metrics = ClusterMetrics.getMetrics();
|
|
|
|
+ assert (metrics != null);
|
|
|
|
+
|
|
|
|
+ //check all 3 nodes joined in as NORMAL
|
|
|
|
+ NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true);
|
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
|
|
+ nodeHeartbeat = nm2.nodeHeartbeat(true);
|
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
|
|
+ nodeHeartbeat = nm3.nodeHeartbeat(true);
|
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+ Assert.assertEquals("All 3 nodes should be active",
|
|
|
|
+ metrics.getNumActiveNMs(), 3);
|
|
|
|
+
|
|
|
|
+ //Remove nm2 from include list, should now be shutdown with timer test
|
|
|
|
+ String ip = NetUtils.normalizeHostName("localhost");
|
|
|
|
+ writeToHostsFile("host1", ip);
|
|
|
|
+ conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH, hostFile
|
|
|
|
+ .getAbsolutePath());
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ nm1.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);
|
|
|
|
+ Assert.assertEquals("Active nodes should be 2",
|
|
|
|
+ metrics.getNumActiveNMs(), 2);
|
|
|
|
+ Assert.assertEquals("Shutdown nodes should be 1",
|
|
|
|
+ metrics.getNumShutdownNMs(), 1);
|
|
|
|
+
|
|
|
|
+ int nodeRemovalTimeout =
|
|
|
|
+ conf.getInt(
|
|
|
|
+ YarnConfiguration.RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC,
|
|
|
|
+ YarnConfiguration.
|
|
|
|
+ DEFAULT_RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC);
|
|
|
|
+ int nodeRemovalInterval =
|
|
|
|
+ rmContext.getNodesListManager().getNodeRemovalCheckInterval();
|
|
|
|
+ long maxThreadSleeptime = nodeRemovalInterval + nodeRemovalTimeout;
|
|
|
|
+ latch.await(maxThreadSleeptime, TimeUnit.MILLISECONDS);
|
|
|
|
+
|
|
|
|
+ rmNode = rmContext.getInactiveRMNodes().get(nm2.getNodeId());
|
|
|
|
+ Assert.assertEquals("Node should have been forgotten!",
|
|
|
|
+ rmNode, null);
|
|
|
|
+ Assert.assertEquals("Shutdown nodes should be 0 now",
|
|
|
|
+ metrics.getNumShutdownNMs(), 0);
|
|
|
|
+
|
|
|
|
+ //Check node removal and re-addition before timer expires
|
|
|
|
+ writeToHostsFile("host1", ip, "host2");
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ nm2 = rm.registerNode("host2:5678", 10240);
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+ writeToHostsFile("host1", ip);
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+ rmNode = rmContext.getInactiveRMNodes().get(nm2.getNodeId());
|
|
|
|
+ Assert.assertEquals("Node should be shutdown",
|
|
|
|
+ rmNode.getState(), NodeState.SHUTDOWN);
|
|
|
|
+ Assert.assertEquals("Active nodes should be 2",
|
|
|
|
+ metrics.getNumActiveNMs(), 2);
|
|
|
|
+ Assert.assertEquals("Shutdown nodes should be 1",
|
|
|
|
+ metrics.getNumShutdownNMs(), 1);
|
|
|
|
+
|
|
|
|
+ //add back the node before timer expires
|
|
|
|
+ latch.await(maxThreadSleeptime - 2000, TimeUnit.MILLISECONDS);
|
|
|
|
+ writeToHostsFile("host1", ip, "host2");
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ nm2 = rm.registerNode("host2:5678", 10240);
|
|
|
|
+ nodeHeartbeat = nm2.nodeHeartbeat(true);
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
|
|
+ Assert.assertEquals("Shutdown nodes should be 0 now",
|
|
|
|
+ metrics.getNumShutdownNMs(), 0);
|
|
|
|
+ Assert.assertEquals("All 3 nodes should be active",
|
|
|
|
+ metrics.getNumActiveNMs(), 3);
|
|
|
|
+
|
|
|
|
+ //Decommission this node, check timer doesn't remove it
|
|
|
|
+ writeToHostsFile("host1", "host2", ip);
|
|
|
|
+ writeToHostsFile(excludeHostFile, "host2");
|
|
|
|
+ conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH, excludeHostFile
|
|
|
|
+ .getAbsolutePath());
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+ rmNode = doGraceful ? rmContext.getRMNodes().get(nm2.getNodeId()) :
|
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId());
|
|
|
|
+ Assert.assertTrue("Node should be DECOMMISSIONED or DECOMMISSIONING",
|
|
|
|
+ (rmNode.getState() == NodeState.DECOMMISSIONED) ||
|
|
|
|
+ (rmNode.getState() == NodeState.DECOMMISSIONING));
|
|
|
|
+ if (rmNode.getState() == NodeState.DECOMMISSIONED) {
|
|
|
|
+ Assert.assertEquals("Decommissioned/ing nodes should be 1 now",
|
|
|
|
+ metrics.getNumDecommisionedNMs(), 1);
|
|
|
|
+ }
|
|
|
|
+ latch.await(maxThreadSleeptime, TimeUnit.MILLISECONDS);
|
|
|
|
+
|
|
|
|
+ rmNode = doGraceful ? rmContext.getRMNodes().get(nm2.getNodeId()) :
|
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId());
|
|
|
|
+ Assert.assertTrue("Node should be DECOMMISSIONED or DECOMMISSIONING",
|
|
|
|
+ (rmNode.getState() == NodeState.DECOMMISSIONED) ||
|
|
|
|
+ (rmNode.getState() == NodeState.DECOMMISSIONING));
|
|
|
|
+ if (rmNode.getState() == NodeState.DECOMMISSIONED) {
|
|
|
|
+ Assert.assertEquals("Decommissioned/ing nodes should be 1 now",
|
|
|
|
+ metrics.getNumDecommisionedNMs(), 1);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ //Test decommed/ing node that transitions to untracked,timer should remove
|
|
|
|
+ writeToHostsFile("host1", ip, "host2");
|
|
|
|
+ writeToHostsFile(excludeHostFile, "host2");
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
|
+ //nm2.nodeHeartbeat(true);
|
|
|
|
+ nm3.nodeHeartbeat(true);
|
|
|
|
+ latch.await(maxThreadSleeptime, TimeUnit.MILLISECONDS);
|
|
|
|
+ rmNode = doGraceful ? rmContext.getRMNodes().get(nm2.getNodeId()) :
|
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId());
|
|
|
|
+ Assert.assertNotEquals("Timer for this node was not canceled!",
|
|
|
|
+ rmNode, null);
|
|
|
|
+ Assert.assertTrue("Node should be DECOMMISSIONED or DECOMMISSIONING",
|
|
|
|
+ (rmNode.getState() == NodeState.DECOMMISSIONED) ||
|
|
|
|
+ (rmNode.getState() == NodeState.DECOMMISSIONING));
|
|
|
|
+
|
|
|
|
+ writeToHostsFile("host1", ip);
|
|
|
|
+ writeToHostsFile(excludeHostFile, "");
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ latch.await(maxThreadSleeptime, TimeUnit.MILLISECONDS);
|
|
|
|
+ rmNode = doGraceful ? rmContext.getRMNodes().get(nm2.getNodeId()) :
|
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId());
|
|
|
|
+ Assert.assertEquals("Node should have been forgotten!",
|
|
|
|
+ rmNode, null);
|
|
|
|
+ Assert.assertEquals("Shutdown nodes should be 0 now",
|
|
|
|
+ metrics.getNumDecommisionedNMs(), 0);
|
|
|
|
+ Assert.assertEquals("Shutdown nodes should be 0 now",
|
|
|
|
+ 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,
|
|
|
|
+ excludeHostFile.getAbsolutePath());
|
|
|
|
+ writeToHostsFile(hostFile, "host1", "localhost", "host2");
|
|
|
|
+ writeToHostsFile(excludeHostFile, "");
|
|
|
|
+ conf.setInt(YarnConfiguration.RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC,
|
|
|
|
+ timeoutValue);
|
|
|
|
+
|
|
|
|
+ rm = new MockRM(conf);
|
|
|
|
+ rm.init(conf);
|
|
|
|
+ rm.start();
|
|
|
|
+ RMContext rmContext = rm.getRMContext();
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ MockNM nm1 = rm.registerNode("host1:1234", 5120);
|
|
|
|
+ MockNM nm2 = rm.registerNode("host2:5678", 10240);
|
|
|
|
+ MockNM nm3 = rm.registerNode("localhost:4433", 1024);
|
|
|
|
+ ClusterMetrics clusterMetrics = ClusterMetrics.getMetrics();
|
|
|
|
+ ClusterMetrics metrics = clusterMetrics;
|
|
|
|
+ assert (metrics != null);
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+ //check all 3 nodes joined in as NORMAL
|
|
|
|
+ NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true);
|
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
|
|
+ nodeHeartbeat = nm2.nodeHeartbeat(true);
|
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
|
|
+ nodeHeartbeat = nm3.nodeHeartbeat(true);
|
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+ Assert.assertEquals("All 3 nodes should be active",
|
|
|
|
+ metrics.getNumActiveNMs(), 3);
|
|
|
|
+ int waitCount = 0;
|
|
|
|
+ while(waitCount ++<20){
|
|
|
|
+ synchronized (this) {
|
|
|
|
+ wait(200);
|
|
|
|
+ }
|
|
|
|
+ nm3.nodeHeartbeat(true);
|
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
|
+ }
|
|
|
|
+ Assert.assertNotEquals("host2 should be a lost NM!",
|
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null);
|
|
|
|
+ Assert.assertEquals("host2 should be a lost NM!",
|
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId()).getState(),
|
|
|
|
+ NodeState.LOST);
|
|
|
|
+ Assert.assertEquals("There should be 1 Lost NM!",
|
|
|
|
+ clusterMetrics.getNumLostNMs(), 1);
|
|
|
|
+ Assert.assertEquals("There should be 2 Active NM!",
|
|
|
|
+ clusterMetrics.getNumActiveNMs(), 2);
|
|
|
|
+ int nodeRemovalTimeout =
|
|
|
|
+ conf.getInt(
|
|
|
|
+ YarnConfiguration.RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC,
|
|
|
|
+ YarnConfiguration.
|
|
|
|
+ DEFAULT_RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC);
|
|
|
|
+ int nodeRemovalInterval =
|
|
|
|
+ rmContext.getNodesListManager().getNodeRemovalCheckInterval();
|
|
|
|
+ long maxThreadSleeptime = nodeRemovalInterval + nodeRemovalTimeout;
|
|
|
|
+ writeToHostsFile(hostFile, "host1", "localhost");
|
|
|
|
+ writeToHostsFile(excludeHostFile, "");
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
|
+ nm3.nodeHeartbeat(true);
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+ waitCount = 0;
|
|
|
|
+ while(rmContext.getInactiveRMNodes().get(
|
|
|
|
+ nm2.getNodeId()) != null && waitCount++ < 2){
|
|
|
|
+ synchronized (this) {
|
|
|
|
+ wait(maxThreadSleeptime);
|
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
|
+ nm2.nodeHeartbeat(true);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ Assert.assertEquals("host2 should have been forgotten!",
|
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null);
|
|
|
|
+ Assert.assertEquals("There should be no Lost NMs!",
|
|
|
|
+ clusterMetrics.getNumLostNMs(), 0);
|
|
|
|
+ Assert.assertEquals("There should be 2 Active NM!",
|
|
|
|
+ clusterMetrics.getNumActiveNMs(), 2);
|
|
|
|
+ rm.stop();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void testNodeRemovalUtilRebooted(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,
|
|
|
|
+ hostFile.getAbsolutePath());
|
|
|
|
+ conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
|
|
|
|
+ excludeHostFile.getAbsolutePath());
|
|
|
|
+ writeToHostsFile(hostFile, "host1", "localhost", "host2");
|
|
|
|
+ writeToHostsFile(excludeHostFile, "");
|
|
|
|
+ conf.setInt(YarnConfiguration.RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC,
|
|
|
|
+ timeoutValue);
|
|
|
|
+
|
|
|
|
+ rm = new MockRM(conf);
|
|
|
|
+ rm.init(conf);
|
|
|
|
+ rm.start();
|
|
|
|
+ RMContext rmContext = rm.getRMContext();
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ MockNM nm1 = rm.registerNode("host1:1234", 5120);
|
|
|
|
+ MockNM nm2 = rm.registerNode("host2:5678", 10240);
|
|
|
|
+ MockNM nm3 = rm.registerNode("localhost:4433", 1024);
|
|
|
|
+ ClusterMetrics clusterMetrics = ClusterMetrics.getMetrics();
|
|
|
|
+ ClusterMetrics metrics = clusterMetrics;
|
|
|
|
+ assert (metrics != null);
|
|
|
|
+ NodeHeartbeatResponse nodeHeartbeat = nm2.nodeHeartbeat(
|
|
|
|
+ new HashMap<ApplicationId, List<ContainerStatus>>(), true, -100);
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+
|
|
|
|
+ Assert.assertNotEquals("host2 should be a rebooted NM!",
|
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null);
|
|
|
|
+ Assert.assertEquals("host2 should be a rebooted NM!",
|
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId()).getState(),
|
|
|
|
+ NodeState.REBOOTED);
|
|
|
|
+ Assert.assertEquals("There should be 1 Rebooted NM!",
|
|
|
|
+ clusterMetrics.getNumRebootedNMs(), 1);
|
|
|
|
+ Assert.assertEquals("There should be 2 Active NM!",
|
|
|
|
+ clusterMetrics.getNumActiveNMs(), 2);
|
|
|
|
+
|
|
|
|
+ int nodeRemovalTimeout =
|
|
|
|
+ conf.getInt(
|
|
|
|
+ YarnConfiguration.RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC,
|
|
|
|
+ YarnConfiguration.
|
|
|
|
+ DEFAULT_RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC);
|
|
|
|
+ int nodeRemovalInterval =
|
|
|
|
+ rmContext.getNodesListManager().getNodeRemovalCheckInterval();
|
|
|
|
+ long maxThreadSleeptime = nodeRemovalInterval + nodeRemovalTimeout;
|
|
|
|
+ writeToHostsFile(hostFile, "host1", "localhost");
|
|
|
|
+ writeToHostsFile(excludeHostFile, "");
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
|
+ nm2.nodeHeartbeat(true);
|
|
|
|
+ nm3.nodeHeartbeat(true);
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+ int waitCount = 0;
|
|
|
|
+ while(rmContext.getInactiveRMNodes().get(
|
|
|
|
+ nm2.getNodeId()) != null && waitCount++ < 2){
|
|
|
|
+ synchronized (this) {
|
|
|
|
+ wait(maxThreadSleeptime);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ Assert.assertEquals("host2 should have been forgotten!",
|
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null);
|
|
|
|
+ Assert.assertEquals("There should be no Rebooted NMs!",
|
|
|
|
+ clusterMetrics.getNumRebootedNMs(), 0);
|
|
|
|
+ Assert.assertEquals("There should be 2 Active NM!",
|
|
|
|
+ clusterMetrics.getNumActiveNMs(), 2);
|
|
|
|
+ rm.stop();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void testNodeRemovalUtilUnhealthy(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,
|
|
|
|
+ hostFile.getAbsolutePath());
|
|
|
|
+ conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
|
|
|
|
+ excludeHostFile.getAbsolutePath());
|
|
|
|
+ writeToHostsFile(hostFile, "host1", "localhost", "host2");
|
|
|
|
+ writeToHostsFile(excludeHostFile, "");
|
|
|
|
+ conf.setInt(YarnConfiguration.RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC,
|
|
|
|
+ timeoutValue);
|
|
|
|
+
|
|
|
|
+ rm = new MockRM(conf);
|
|
|
|
+ rm.init(conf);
|
|
|
|
+ rm.start();
|
|
|
|
+ RMContext rmContext = rm.getRMContext();
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ MockNM nm1 = rm.registerNode("host1:1234", 5120);
|
|
|
|
+ MockNM nm2 = rm.registerNode("host2:5678", 10240);
|
|
|
|
+ MockNM nm3 = rm.registerNode("localhost:4433", 1024);
|
|
|
|
+ ClusterMetrics clusterMetrics = ClusterMetrics.getMetrics();
|
|
|
|
+ ClusterMetrics metrics = clusterMetrics;
|
|
|
|
+ assert (metrics != null);
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+ //check all 3 nodes joined in as NORMAL
|
|
|
|
+ NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true);
|
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
|
|
+ nodeHeartbeat = nm2.nodeHeartbeat(true);
|
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
|
|
+ nodeHeartbeat = nm3.nodeHeartbeat(true);
|
|
|
|
+ Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
|
|
|
|
+ rm.drainEvents();
|
|
|
|
+ Assert.assertEquals("All 3 nodes should be active",
|
|
|
|
+ metrics.getNumActiveNMs(), 3);
|
|
|
|
+ // node healthy
|
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
|
+ nm2.nodeHeartbeat(false);
|
|
|
|
+ nm3.nodeHeartbeat(true);
|
|
|
|
+ checkUnhealthyNMCount(rm, nm2, true, 1);
|
|
|
|
+ writeToHostsFile(hostFile, "host1", "localhost");
|
|
|
|
+ writeToHostsFile(excludeHostFile, "");
|
|
|
|
+ refreshNodesOption(doGraceful, conf);
|
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
|
+ 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);
|
|
|
|
+ Assert.assertEquals("There should be 2 Active NM!",
|
|
|
|
+ clusterMetrics.getNumActiveNMs(), 2);
|
|
|
|
+ Assert.assertEquals("There should be 1 Shutdown NM!",
|
|
|
|
+ clusterMetrics.getNumShutdownNMs(), 1);
|
|
|
|
+ Assert.assertEquals("There should be 0 Unhealthy NM!",
|
|
|
|
+ clusterMetrics.getUnhealthyNMs(), 0);
|
|
|
|
+ int nodeRemovalTimeout =
|
|
|
|
+ conf.getInt(
|
|
|
|
+ YarnConfiguration.RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC,
|
|
|
|
+ YarnConfiguration.
|
|
|
|
+ DEFAULT_RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC);
|
|
|
|
+ int nodeRemovalInterval =
|
|
|
|
+ rmContext.getNodesListManager().getNodeRemovalCheckInterval();
|
|
|
|
+ long maxThreadSleeptime = nodeRemovalInterval + nodeRemovalTimeout;
|
|
|
|
+ int waitCount = 0;
|
|
|
|
+ while(rmContext.getInactiveRMNodes().get(
|
|
|
|
+ nm2.getNodeId()) != null && waitCount++ < 2){
|
|
|
|
+ synchronized (this) {
|
|
|
|
+ wait(maxThreadSleeptime);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ Assert.assertEquals("host2 should have been forgotten!",
|
|
|
|
+ rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null);
|
|
|
|
+ Assert.assertEquals("There should be no Shutdown NMs!",
|
|
|
|
+ clusterMetrics.getNumRebootedNMs(), 0);
|
|
|
|
+ Assert.assertEquals("There should be 2 Active NM!",
|
|
|
|
+ clusterMetrics.getNumActiveNMs(), 2);
|
|
|
|
+ rm.stop();
|
|
|
|
+ }
|
|
|
|
+
|
|
private void writeToHostsFile(String... hosts) throws IOException {
|
|
private void writeToHostsFile(String... hosts) throws IOException {
|
|
writeToHostsFile(hostFile, hosts);
|
|
writeToHostsFile(hostFile, hosts);
|
|
}
|
|
}
|