|
@@ -19,11 +19,11 @@
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager;
|
|
|
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Collections;
|
|
|
import java.util.List;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
-import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
@@ -35,7 +35,8 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.DecommissioningNodesWatcher.DecommissioningNodeStatus;
|
|
|
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.rmnode.RMNode;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
@@ -58,38 +59,106 @@ public class TestDecommissioningNodesWatcher {
|
|
|
new DecommissioningNodesWatcher(rm.getRMContext());
|
|
|
|
|
|
MockNM nm1 = rm.registerNode("host1:1234", 10240);
|
|
|
- RMNode node1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
|
|
|
+ RMNodeImpl node1 =
|
|
|
+ (RMNodeImpl) rm.getRMContext().getRMNodes().get(nm1.getNodeId());
|
|
|
NodeId id1 = nm1.getNodeId();
|
|
|
|
|
|
rm.waitForState(id1, NodeState.RUNNING);
|
|
|
- Assert.assertFalse(watcher.checkReadyToBeDecommissioned(id1));
|
|
|
|
|
|
RMApp app = rm.submitApp(2000);
|
|
|
MockAM am = MockRM.launchAndRegisterAM(app, rm, nm1);
|
|
|
|
|
|
+ NodeStatus nodeStatus = createNodeStatus(id1, app, 3);
|
|
|
+ node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
|
|
|
+
|
|
|
// Setup nm1 as DECOMMISSIONING for DecommissioningNodesWatcher.
|
|
|
rm.sendNodeGracefulDecommission(nm1,
|
|
|
YarnConfiguration.DEFAULT_RM_NODE_GRACEFUL_DECOMMISSION_TIMEOUT);
|
|
|
rm.waitForState(id1, NodeState.DECOMMISSIONING);
|
|
|
|
|
|
// Update status with decreasing number of running containers until 0.
|
|
|
- watcher.update(node1, createNodeStatus(id1, app, 12));
|
|
|
- watcher.update(node1, createNodeStatus(id1, app, 11));
|
|
|
+ nodeStatus = createNodeStatus(id1, app, 3);
|
|
|
+ node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
|
|
|
+ watcher.update(node1, nodeStatus);
|
|
|
+
|
|
|
+ nodeStatus = createNodeStatus(id1, app, 2);
|
|
|
+ node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
|
|
|
+ watcher.update(node1, nodeStatus);
|
|
|
Assert.assertFalse(watcher.checkReadyToBeDecommissioned(id1));
|
|
|
|
|
|
- watcher.update(node1, createNodeStatus(id1, app, 1));
|
|
|
+ nodeStatus = createNodeStatus(id1, app, 1);
|
|
|
+ node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
|
|
|
+ watcher.update(node1, nodeStatus);
|
|
|
Assert.assertEquals(DecommissioningNodeStatus.WAIT_CONTAINER,
|
|
|
- watcher.checkDecommissioningStatus(id1));
|
|
|
+ watcher.checkDecommissioningStatus(id1));
|
|
|
+
|
|
|
+ nodeStatus = createNodeStatus(id1, app, 0);
|
|
|
+ watcher.update(node1, nodeStatus);
|
|
|
+ node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
|
|
|
+ Assert.assertEquals(DecommissioningNodeStatus.WAIT_APP,
|
|
|
+ watcher.checkDecommissioningStatus(id1));
|
|
|
+
|
|
|
+ // Set app to be FINISHED and verified DecommissioningNodeStatus is READY.
|
|
|
+ MockRM.finishAMAndVerifyAppState(app, rm, nm1, am);
|
|
|
+ rm.waitForState(app.getApplicationId(), RMAppState.FINISHED);
|
|
|
+ watcher.update(node1, nodeStatus);
|
|
|
+ Assert.assertEquals(DecommissioningNodeStatus.READY,
|
|
|
+ watcher.checkDecommissioningStatus(id1));
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testDecommissioningNodesWatcherWithPreviousRunningApps()
|
|
|
+ throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.set(YarnConfiguration.RM_NODE_GRACEFUL_DECOMMISSION_TIMEOUT, "40");
|
|
|
+
|
|
|
+ rm = new MockRM(conf);
|
|
|
+ rm.start();
|
|
|
+
|
|
|
+ DecommissioningNodesWatcher watcher =
|
|
|
+ new DecommissioningNodesWatcher(rm.getRMContext());
|
|
|
+
|
|
|
+ MockNM nm1 = rm.registerNode("host1:1234", 10240);
|
|
|
+ RMNodeImpl node1 =
|
|
|
+ (RMNodeImpl) rm.getRMContext().getRMNodes().get(nm1.getNodeId());
|
|
|
+ NodeId id1 = nm1.getNodeId();
|
|
|
+
|
|
|
+ rm.waitForState(id1, NodeState.RUNNING);
|
|
|
+
|
|
|
+ RMApp app = rm.submitApp(2000);
|
|
|
+ MockAM am = MockRM.launchAndRegisterAM(app, rm, nm1);
|
|
|
|
|
|
- watcher.update(node1, createNodeStatus(id1, app, 0));
|
|
|
+ NodeStatus nodeStatus = createNodeStatus(id1, app, 3);
|
|
|
+ node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
|
|
|
+
|
|
|
+ Assert.assertEquals(1, node1.getRunningApps().size());
|
|
|
+
|
|
|
+ // update node with 0 running containers
|
|
|
+ nodeStatus = createNodeStatus(id1, app, 0);
|
|
|
+ node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
|
|
|
+
|
|
|
+ Assert.assertEquals(1, node1.getRunningApps().size());
|
|
|
+
|
|
|
+ // Setup nm1 as DECOMMISSIONING for DecommissioningNodesWatcher. Right now
|
|
|
+ // there is no container running on the node.
|
|
|
+ rm.sendNodeGracefulDecommission(nm1,
|
|
|
+ YarnConfiguration.DEFAULT_RM_NODE_GRACEFUL_DECOMMISSION_TIMEOUT);
|
|
|
+ rm.waitForState(id1, NodeState.DECOMMISSIONING);
|
|
|
+
|
|
|
+ // we should still get WAIT_APP as container for a running app previously
|
|
|
+ // ran on this node.
|
|
|
+ watcher.update(node1, nodeStatus);
|
|
|
+ Assert.assertFalse(watcher.checkReadyToBeDecommissioned(id1));
|
|
|
Assert.assertEquals(DecommissioningNodeStatus.WAIT_APP,
|
|
|
- watcher.checkDecommissioningStatus(id1));
|
|
|
+ watcher.checkDecommissioningStatus(id1));
|
|
|
|
|
|
// Set app to be FINISHED and verified DecommissioningNodeStatus is READY.
|
|
|
MockRM.finishAMAndVerifyAppState(app, rm, nm1, am);
|
|
|
rm.waitForState(app.getApplicationId(), RMAppState.FINISHED);
|
|
|
+ Assert.assertEquals(0, node1.getRunningApps().size());
|
|
|
+ watcher.update(node1, nodeStatus);
|
|
|
Assert.assertEquals(DecommissioningNodeStatus.READY,
|
|
|
- watcher.checkDecommissioningStatus(id1));
|
|
|
+ watcher.checkDecommissioningStatus(id1));
|
|
|
}
|
|
|
|
|
|
@After
|
|
@@ -103,7 +172,7 @@ public class TestDecommissioningNodesWatcher {
|
|
|
NodeId nodeId, RMApp app, int numRunningContainers) {
|
|
|
return NodeStatus.newInstance(
|
|
|
nodeId, 0, getContainerStatuses(app, numRunningContainers),
|
|
|
- new ArrayList<ApplicationId>(),
|
|
|
+ Collections.emptyList(),
|
|
|
NodeHealthStatus.newInstance(
|
|
|
true, "", System.currentTimeMillis() - 1000),
|
|
|
null, null, null);
|
|
@@ -113,8 +182,8 @@ public class TestDecommissioningNodesWatcher {
|
|
|
// where numRunningContainers are RUNNING.
|
|
|
private List<ContainerStatus> getContainerStatuses(
|
|
|
RMApp app, int numRunningContainers) {
|
|
|
- // Total 12 containers
|
|
|
- final int total = 12;
|
|
|
+ // Total 3 containers
|
|
|
+ final int total = 3;
|
|
|
numRunningContainers = Math.min(total, numRunningContainers);
|
|
|
List<ContainerStatus> output = new ArrayList<ContainerStatus>();
|
|
|
for (int i = 0; i < total; i++) {
|
|
@@ -122,8 +191,8 @@ public class TestDecommissioningNodesWatcher {
|
|
|
ContainerState.COMPLETE : ContainerState.RUNNING;
|
|
|
output.add(ContainerStatus.newInstance(
|
|
|
ContainerId.newContainerId(
|
|
|
- ApplicationAttemptId.newInstance(app.getApplicationId(), i), 1),
|
|
|
- cstate, "Dummy", 0));
|
|
|
+ ApplicationAttemptId.newInstance(app.getApplicationId(), 0), i),
|
|
|
+ cstate, "", 0));
|
|
|
}
|
|
|
return output;
|
|
|
}
|