|
@@ -31,6 +31,7 @@ import java.util.Collections;
|
|
|
import java.util.List;
|
|
|
|
|
|
import org.apache.hadoop.util.HostsFileReader;
|
|
|
+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;
|
|
@@ -43,6 +44,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.event.InlineDispatcher;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
|
|
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
|
|
@@ -105,8 +107,9 @@ public class TestRMNodeTransitions {
|
|
|
InlineDispatcher rmDispatcher = new InlineDispatcher();
|
|
|
|
|
|
rmContext =
|
|
|
- new RMContextImpl(rmDispatcher, null, null, null,
|
|
|
- mock(DelegationTokenRenewer.class), null, null, null, null, null);
|
|
|
+ new RMContextImpl(rmDispatcher, mock(ContainerAllocationExpirer.class),
|
|
|
+ null, null, mock(DelegationTokenRenewer.class), null, null, null,
|
|
|
+ null, null);
|
|
|
NodesListManager nodesListManager = mock(NodesListManager.class);
|
|
|
HostsFileReader reader = mock(HostsFileReader.class);
|
|
|
when(nodesListManager.getHostsReader()).thenReturn(reader);
|
|
@@ -147,7 +150,8 @@ public class TestRMNodeTransitions {
|
|
|
public void tearDown() throws Exception {
|
|
|
}
|
|
|
|
|
|
- private RMNodeStatusEvent getMockRMNodeStatusEvent() {
|
|
|
+ private RMNodeStatusEvent getMockRMNodeStatusEvent(
|
|
|
+ List<ContainerStatus> containerStatus) {
|
|
|
NodeHeartbeatResponse response = mock(NodeHeartbeatResponse.class);
|
|
|
|
|
|
NodeHealthStatus healthStatus = mock(NodeHealthStatus.class);
|
|
@@ -158,6 +162,9 @@ public class TestRMNodeTransitions {
|
|
|
doReturn(healthStatus).when(event).getNodeHealthStatus();
|
|
|
doReturn(response).when(event).getLatestResponse();
|
|
|
doReturn(RMNodeEventType.STATUS_UPDATE).when(event).getType();
|
|
|
+ if (containerStatus != null) {
|
|
|
+ doReturn(containerStatus).when(event).getContainers();
|
|
|
+ }
|
|
|
return event;
|
|
|
}
|
|
|
|
|
@@ -176,7 +183,7 @@ public class TestRMNodeTransitions {
|
|
|
|
|
|
// Now verify that scheduler isn't notified of an expired container
|
|
|
// by checking number of 'completedContainers' it got in the previous event
|
|
|
- RMNodeStatusEvent statusEvent = getMockRMNodeStatusEvent();
|
|
|
+ RMNodeStatusEvent statusEvent = getMockRMNodeStatusEvent(null);
|
|
|
ContainerStatus containerStatus = mock(ContainerStatus.class);
|
|
|
doReturn(completedContainerId).when(containerStatus).getContainerId();
|
|
|
doReturn(Collections.singletonList(containerStatus)).
|
|
@@ -207,11 +214,11 @@ public class TestRMNodeTransitions {
|
|
|
ContainerId completedContainerIdFromNode2_2 = BuilderUtils.newContainerId(
|
|
|
BuilderUtils.newApplicationAttemptId(
|
|
|
BuilderUtils.newApplicationId(1, 1), 1), 2);
|
|
|
-
|
|
|
- RMNodeStatusEvent statusEventFromNode1 = getMockRMNodeStatusEvent();
|
|
|
- RMNodeStatusEvent statusEventFromNode2_1 = getMockRMNodeStatusEvent();
|
|
|
- RMNodeStatusEvent statusEventFromNode2_2 = getMockRMNodeStatusEvent();
|
|
|
-
|
|
|
+
|
|
|
+ RMNodeStatusEvent statusEventFromNode1 = getMockRMNodeStatusEvent(null);
|
|
|
+ RMNodeStatusEvent statusEventFromNode2_1 = getMockRMNodeStatusEvent(null);
|
|
|
+ RMNodeStatusEvent statusEventFromNode2_2 = getMockRMNodeStatusEvent(null);
|
|
|
+
|
|
|
ContainerStatus containerStatusFromNode1 = mock(ContainerStatus.class);
|
|
|
ContainerStatus containerStatusFromNode2_1 = mock(ContainerStatus.class);
|
|
|
ContainerStatus containerStatusFromNode2_2 = mock(ContainerStatus.class);
|
|
@@ -263,8 +270,8 @@ public class TestRMNodeTransitions {
|
|
|
BuilderUtils.newApplicationAttemptId(
|
|
|
BuilderUtils.newApplicationId(1, 1), 1), 1);
|
|
|
|
|
|
- RMNodeStatusEvent statusEvent1 = getMockRMNodeStatusEvent();
|
|
|
- RMNodeStatusEvent statusEvent2 = getMockRMNodeStatusEvent();
|
|
|
+ RMNodeStatusEvent statusEvent1 = getMockRMNodeStatusEvent(null);
|
|
|
+ RMNodeStatusEvent statusEvent2 = getMockRMNodeStatusEvent(null);
|
|
|
|
|
|
ContainerStatus containerStatus1 = mock(ContainerStatus.class);
|
|
|
ContainerStatus containerStatus2 = mock(ContainerStatus.class);
|
|
@@ -499,7 +506,7 @@ public class TestRMNodeTransitions {
|
|
|
|
|
|
// Verify status update does not clear containers/apps to cleanup
|
|
|
// but updating heartbeat response for cleanup does
|
|
|
- RMNodeStatusEvent statusEvent = getMockRMNodeStatusEvent();
|
|
|
+ RMNodeStatusEvent statusEvent = getMockRMNodeStatusEvent(null);
|
|
|
node.handle(statusEvent);
|
|
|
Assert.assertEquals(1, node.getContainersToCleanUp().size());
|
|
|
Assert.assertEquals(1, node.getAppsToCleanup().size());
|
|
@@ -706,4 +713,35 @@ public class TestRMNodeTransitions {
|
|
|
null, null));
|
|
|
Assert.assertEquals(nmVersion2, node.getNodeManagerVersion());
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testContainerExpire() throws Exception {
|
|
|
+ ContainerAllocationExpirer mockExpirer =
|
|
|
+ mock(ContainerAllocationExpirer.class);
|
|
|
+ ApplicationId appId =
|
|
|
+ ApplicationId.newInstance(System.currentTimeMillis(), 1);
|
|
|
+ ApplicationAttemptId appAttemptId =
|
|
|
+ ApplicationAttemptId.newInstance(appId, 1);
|
|
|
+ ContainerId containerId1 = ContainerId.newContainerId(appAttemptId, 1L);
|
|
|
+ ContainerId containerId2 = ContainerId.newContainerId(appAttemptId, 2L);
|
|
|
+ mockExpirer.register(containerId1);
|
|
|
+ mockExpirer.register(containerId2);
|
|
|
+ verify(mockExpirer).register(containerId1);
|
|
|
+ verify(mockExpirer).register(containerId2);
|
|
|
+ ((RMContextImpl) rmContext).setContainerAllocationExpirer(mockExpirer);
|
|
|
+ RMNodeImpl rmNode = getRunningNode();
|
|
|
+ ContainerStatus status1 =
|
|
|
+ ContainerStatus
|
|
|
+ .newInstance(containerId1, ContainerState.RUNNING, "", 0);
|
|
|
+ ContainerStatus status2 =
|
|
|
+ ContainerStatus.newInstance(containerId2, ContainerState.COMPLETE, "",
|
|
|
+ 0);
|
|
|
+ List<ContainerStatus> statusList = new ArrayList<ContainerStatus>();
|
|
|
+ statusList.add(status1);
|
|
|
+ statusList.add(status2);
|
|
|
+ RMNodeStatusEvent statusEvent = getMockRMNodeStatusEvent(statusList);
|
|
|
+ rmNode.handle(statusEvent);
|
|
|
+ verify(mockExpirer).unregister(containerId1);
|
|
|
+ verify(mockExpirer).unregister(containerId2);
|
|
|
+ }
|
|
|
}
|