|
@@ -34,7 +34,6 @@ import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.concurrent.ConcurrentMap;
|
|
|
-import java.util.concurrent.ConcurrentSkipListMap;
|
|
|
import java.util.concurrent.CyclicBarrier;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
@@ -117,8 +116,6 @@ public class TestNodeStatusUpdater {
|
|
|
private boolean triggered = false;
|
|
|
private Configuration conf;
|
|
|
private NodeManager nm;
|
|
|
- private boolean containerStatusBackupSuccessfully = true;
|
|
|
- private List<ContainerStatus> completedContainerStatusList = new ArrayList<ContainerStatus>();
|
|
|
private AtomicBoolean assertionFailedInThread = new AtomicBoolean(false);
|
|
|
|
|
|
@Before
|
|
@@ -304,6 +301,8 @@ public class TestNodeStatusUpdater {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ // Test NodeStatusUpdater sends the right container statuses each time it
|
|
|
+ // heart beats.
|
|
|
private class MyNodeStatusUpdater2 extends NodeStatusUpdaterImpl {
|
|
|
public ResourceTracker resourceTracker;
|
|
|
|
|
@@ -555,6 +554,8 @@ public class TestNodeStatusUpdater {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ // Test NodeStatusUpdater sends the right container statuses each time it
|
|
|
+ // heart beats.
|
|
|
private class MyResourceTracker4 implements ResourceTracker {
|
|
|
|
|
|
public NodeAction registerNodeAction = NodeAction.NORMAL;
|
|
@@ -567,10 +568,9 @@ public class TestNodeStatusUpdater {
|
|
|
|
|
|
@Override
|
|
|
public RegisterNodeManagerResponse registerNodeManager(
|
|
|
- RegisterNodeManagerRequest request) throws YarnException,
|
|
|
- IOException {
|
|
|
- RegisterNodeManagerResponse response = recordFactory
|
|
|
- .newRecordInstance(RegisterNodeManagerResponse.class);
|
|
|
+ RegisterNodeManagerRequest request) throws YarnException, IOException {
|
|
|
+ RegisterNodeManagerResponse response =
|
|
|
+ recordFactory.newRecordInstance(RegisterNodeManagerResponse.class);
|
|
|
response.setNodeAction(registerNodeAction);
|
|
|
response.setContainerTokenMasterKey(createMasterKey());
|
|
|
response.setNMTokenMasterKey(createMasterKey());
|
|
@@ -583,67 +583,88 @@ public class TestNodeStatusUpdater {
|
|
|
try {
|
|
|
if (heartBeatID == 0) {
|
|
|
Assert.assertEquals(request.getNodeStatus().getContainersStatuses()
|
|
|
- .size(), 0);
|
|
|
+ .size(), 0);
|
|
|
Assert.assertEquals(context.getContainers().size(), 0);
|
|
|
} else if (heartBeatID == 1) {
|
|
|
- Assert.assertEquals(request.getNodeStatus().getContainersStatuses()
|
|
|
- .size(), 5);
|
|
|
- Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
|
|
|
- .get(0).getState() == ContainerState.RUNNING
|
|
|
- && request.getNodeStatus().getContainersStatuses().get(0)
|
|
|
- .getContainerId().getId() == 1);
|
|
|
- Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
|
|
|
- .get(1).getState() == ContainerState.RUNNING
|
|
|
- && request.getNodeStatus().getContainersStatuses().get(1)
|
|
|
- .getContainerId().getId() == 2);
|
|
|
- Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
|
|
|
- .get(2).getState() == ContainerState.COMPLETE
|
|
|
- && request.getNodeStatus().getContainersStatuses().get(2)
|
|
|
- .getContainerId().getId() == 3);
|
|
|
- Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
|
|
|
- .get(3).getState() == ContainerState.COMPLETE
|
|
|
- && request.getNodeStatus().getContainersStatuses().get(3)
|
|
|
- .getContainerId().getId() == 4);
|
|
|
- Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
|
|
|
- .get(4).getState() == ContainerState.RUNNING
|
|
|
- && request.getNodeStatus().getContainersStatuses().get(4)
|
|
|
- .getContainerId().getId() == 5);
|
|
|
- throw new java.net.ConnectException("Lost the heartbeat response");
|
|
|
+ List<ContainerStatus> statuses =
|
|
|
+ request.getNodeStatus().getContainersStatuses();
|
|
|
+ Assert.assertEquals(statuses.size(), 2);
|
|
|
+ Assert.assertEquals(context.getContainers().size(), 2);
|
|
|
+
|
|
|
+ ContainerStatus containerStatus2 =
|
|
|
+ createContainerStatus(2, ContainerState.RUNNING);
|
|
|
+ ContainerStatus containerStatus3 =
|
|
|
+ createContainerStatus(3, ContainerState.COMPLETE);
|
|
|
+ boolean container2Exist = false, container3Exist = false;
|
|
|
+ for (ContainerStatus status : statuses) {
|
|
|
+ if (status.getContainerId().equals(
|
|
|
+ containerStatus2.getContainerId())) {
|
|
|
+ Assert.assertTrue(status.getState().equals(
|
|
|
+ containerStatus2.getState()));
|
|
|
+ container2Exist = true;
|
|
|
+ }
|
|
|
+ if (status.getContainerId().equals(
|
|
|
+ containerStatus3.getContainerId())) {
|
|
|
+ Assert.assertTrue(status.getState().equals(
|
|
|
+ containerStatus3.getState()));
|
|
|
+ container3Exist = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ Assert.assertTrue(container2Exist && container3Exist);
|
|
|
+
|
|
|
+ // should throw exception that can be retried by the
|
|
|
+ // nodeStatusUpdaterRunnable, otherwise nm just shuts down and the
|
|
|
+ // test passes.
|
|
|
+ throw new YarnRuntimeException("Lost the heartbeat response");
|
|
|
} else if (heartBeatID == 2) {
|
|
|
- Assert.assertEquals(request.getNodeStatus().getContainersStatuses()
|
|
|
- .size(), 7);
|
|
|
- Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
|
|
|
- .get(0).getState() == ContainerState.COMPLETE
|
|
|
- && request.getNodeStatus().getContainersStatuses().get(0)
|
|
|
- .getContainerId().getId() == 3);
|
|
|
- Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
|
|
|
- .get(1).getState() == ContainerState.COMPLETE
|
|
|
- && request.getNodeStatus().getContainersStatuses().get(1)
|
|
|
- .getContainerId().getId() == 4);
|
|
|
- Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
|
|
|
- .get(2).getState() == ContainerState.RUNNING
|
|
|
- && request.getNodeStatus().getContainersStatuses().get(2)
|
|
|
- .getContainerId().getId() == 1);
|
|
|
- Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
|
|
|
- .get(3).getState() == ContainerState.RUNNING
|
|
|
- && request.getNodeStatus().getContainersStatuses().get(3)
|
|
|
- .getContainerId().getId() == 2);
|
|
|
- Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
|
|
|
- .get(4).getState() == ContainerState.RUNNING
|
|
|
- && request.getNodeStatus().getContainersStatuses().get(4)
|
|
|
- .getContainerId().getId() == 5);
|
|
|
- Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
|
|
|
- .get(5).getState() == ContainerState.RUNNING
|
|
|
- && request.getNodeStatus().getContainersStatuses().get(5)
|
|
|
- .getContainerId().getId() == 6);
|
|
|
- Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
|
|
|
- .get(6).getState() == ContainerState.COMPLETE
|
|
|
- && request.getNodeStatus().getContainersStatuses().get(6)
|
|
|
- .getContainerId().getId() == 7);
|
|
|
+ List<ContainerStatus> statuses =
|
|
|
+ request.getNodeStatus().getContainersStatuses();
|
|
|
+ Assert.assertEquals(statuses.size(), 4);
|
|
|
+ Assert.assertEquals(context.getContainers().size(), 4);
|
|
|
+
|
|
|
+ ContainerStatus containerStatus2 =
|
|
|
+ createContainerStatus(2, ContainerState.RUNNING);
|
|
|
+ ContainerStatus containerStatus3 =
|
|
|
+ createContainerStatus(3, ContainerState.COMPLETE);
|
|
|
+ ContainerStatus containerStatus4 =
|
|
|
+ createContainerStatus(4, ContainerState.RUNNING);
|
|
|
+ ContainerStatus containerStatus5 =
|
|
|
+ createContainerStatus(5, ContainerState.COMPLETE);
|
|
|
+
|
|
|
+ boolean container2Exist = false, container3Exist = false, container4Exist =
|
|
|
+ false, container5Exist = false;
|
|
|
+ for (ContainerStatus status : statuses) {
|
|
|
+ if (status.getContainerId().equals(
|
|
|
+ containerStatus2.getContainerId())) {
|
|
|
+ Assert.assertTrue(status.getState().equals(
|
|
|
+ containerStatus2.getState()));
|
|
|
+ container2Exist = true;
|
|
|
+ }
|
|
|
+ if (status.getContainerId().equals(
|
|
|
+ containerStatus3.getContainerId())) {
|
|
|
+ Assert.assertTrue(status.getState().equals(
|
|
|
+ containerStatus3.getState()));
|
|
|
+ container3Exist = true;
|
|
|
+ }
|
|
|
+ if (status.getContainerId().equals(
|
|
|
+ containerStatus4.getContainerId())) {
|
|
|
+ Assert.assertTrue(status.getState().equals(
|
|
|
+ containerStatus4.getState()));
|
|
|
+ container4Exist = true;
|
|
|
+ }
|
|
|
+ if (status.getContainerId().equals(
|
|
|
+ containerStatus5.getContainerId())) {
|
|
|
+ Assert.assertTrue(status.getState().equals(
|
|
|
+ containerStatus5.getState()));
|
|
|
+ container5Exist = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ Assert.assertTrue(container2Exist && container3Exist
|
|
|
+ && container4Exist && container5Exist);
|
|
|
}
|
|
|
} catch (AssertionError error) {
|
|
|
- LOG.info(error);
|
|
|
- containerStatusBackupSuccessfully = false;
|
|
|
+ error.printStackTrace();
|
|
|
+ assertionFailedInThread.set(true);
|
|
|
} finally {
|
|
|
heartBeatID++;
|
|
|
}
|
|
@@ -651,9 +672,7 @@ public class TestNodeStatusUpdater {
|
|
|
nodeStatus.setResponseId(heartBeatID);
|
|
|
NodeHeartbeatResponse nhResponse =
|
|
|
YarnServerBuilderUtils.newNodeHeartbeatResponse(heartBeatID,
|
|
|
- heartBeatNodeAction,
|
|
|
- null, null, null,
|
|
|
- null, 1000L);
|
|
|
+ heartBeatNodeAction, null, null, null, null, 1000L);
|
|
|
return nhResponse;
|
|
|
}
|
|
|
}
|
|
@@ -761,7 +780,7 @@ public class TestNodeStatusUpdater {
|
|
|
ContainerId cId = ContainerId.newInstance(appAttemptId, 0);
|
|
|
|
|
|
|
|
|
- nodeStatusUpdater.addStoppedContainersToCache(cId);
|
|
|
+ nodeStatusUpdater.updateStoppedContainersInCache(cId);
|
|
|
Assert.assertTrue(nodeStatusUpdater.isContainerRecentlyStopped(cId));
|
|
|
|
|
|
long time1 = System.currentTimeMillis();
|
|
@@ -1119,7 +1138,8 @@ public class TestNodeStatusUpdater {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Test completed containerStatus get back up when heart beat lost
|
|
|
+ * Test completed containerStatus get back up when heart beat lost, and will
|
|
|
+ * be sent via next heart beat.
|
|
|
*/
|
|
|
@Test(timeout = 200000)
|
|
|
public void testCompletedContainerStatusBackup() throws Exception {
|
|
@@ -1150,7 +1170,7 @@ public class TestNodeStatusUpdater {
|
|
|
while (heartBeatID <= 3 && waitCount++ != 20) {
|
|
|
Thread.sleep(500);
|
|
|
}
|
|
|
- if(!containerStatusBackupSuccessfully) {
|
|
|
+ if(assertionFailedInThread.get()) {
|
|
|
Assert.fail("ContainerStatus Backup failed");
|
|
|
}
|
|
|
nm.stop();
|
|
@@ -1239,9 +1259,8 @@ public class TestNodeStatusUpdater {
|
|
|
nm.stop();
|
|
|
}
|
|
|
|
|
|
+ // Add new containers info into NM context each time node heart beats.
|
|
|
private class MyNMContext extends NMContext {
|
|
|
- ConcurrentMap<ContainerId, Container> containers =
|
|
|
- new ConcurrentSkipListMap<ContainerId, Container>();
|
|
|
|
|
|
public MyNMContext(
|
|
|
NMContainerTokenSecretManager containerTokenSecretManager,
|
|
@@ -1254,11 +1273,6 @@ public class TestNodeStatusUpdater {
|
|
|
if (heartBeatID == 0) {
|
|
|
return containers;
|
|
|
} else if (heartBeatID == 1) {
|
|
|
- ContainerStatus containerStatus1 =
|
|
|
- createContainerStatus(1, ContainerState.RUNNING);
|
|
|
- Container container1 = getMockContainer(containerStatus1);
|
|
|
- containers.put(containerStatus1.getContainerId(), container1);
|
|
|
-
|
|
|
ContainerStatus containerStatus2 =
|
|
|
createContainerStatus(2, ContainerState.RUNNING);
|
|
|
Container container2 = getMockContainer(containerStatus2);
|
|
@@ -1268,60 +1282,45 @@ public class TestNodeStatusUpdater {
|
|
|
createContainerStatus(3, ContainerState.COMPLETE);
|
|
|
Container container3 = getMockContainer(containerStatus3);
|
|
|
containers.put(containerStatus3.getContainerId(), container3);
|
|
|
- completedContainerStatusList.add(containerStatus3);
|
|
|
-
|
|
|
+ return containers;
|
|
|
+ } else if (heartBeatID == 2) {
|
|
|
ContainerStatus containerStatus4 =
|
|
|
- createContainerStatus(4, ContainerState.COMPLETE);
|
|
|
+ createContainerStatus(4, ContainerState.RUNNING);
|
|
|
Container container4 = getMockContainer(containerStatus4);
|
|
|
containers.put(containerStatus4.getContainerId(), container4);
|
|
|
- completedContainerStatusList.add(containerStatus4);
|
|
|
|
|
|
ContainerStatus containerStatus5 =
|
|
|
- createContainerStatus(5, ContainerState.RUNNING);
|
|
|
+ createContainerStatus(5, ContainerState.COMPLETE);
|
|
|
Container container5 = getMockContainer(containerStatus5);
|
|
|
containers.put(containerStatus5.getContainerId(), container5);
|
|
|
-
|
|
|
- return containers;
|
|
|
- } else if (heartBeatID == 2) {
|
|
|
- ContainerStatus containerStatus6 =
|
|
|
- createContainerStatus(6, ContainerState.RUNNING);
|
|
|
- Container container6 = getMockContainer(containerStatus6);
|
|
|
- containers.put(containerStatus6.getContainerId(), container6);
|
|
|
-
|
|
|
- ContainerStatus containerStatus7 =
|
|
|
- createContainerStatus(7, ContainerState.COMPLETE);
|
|
|
- Container container7 = getMockContainer(containerStatus7);
|
|
|
- containers.put(containerStatus7.getContainerId(), container7);
|
|
|
- completedContainerStatusList.add(containerStatus7);
|
|
|
-
|
|
|
return containers;
|
|
|
} else {
|
|
|
containers.clear();
|
|
|
-
|
|
|
return containers;
|
|
|
}
|
|
|
}
|
|
|
+ }
|
|
|
|
|
|
- private ContainerStatus createContainerStatus(int id,
|
|
|
- ContainerState containerState) {
|
|
|
- ApplicationId applicationId =
|
|
|
- BuilderUtils.newApplicationId(System.currentTimeMillis(), id);
|
|
|
- ApplicationAttemptId applicationAttemptId =
|
|
|
- BuilderUtils.newApplicationAttemptId(applicationId, id);
|
|
|
- ContainerId contaierId =
|
|
|
- BuilderUtils.newContainerId(applicationAttemptId, id);
|
|
|
- ContainerStatus containerStatus =
|
|
|
- BuilderUtils.newContainerStatus(contaierId, containerState,
|
|
|
- "test_containerStatus: id=" + id + ", containerState: "
|
|
|
- + containerState, 0);
|
|
|
- return containerStatus;
|
|
|
- }
|
|
|
+ public static ContainerStatus createContainerStatus(int id,
|
|
|
+ ContainerState containerState) {
|
|
|
+ ApplicationId applicationId = ApplicationId.newInstance(0, 1);
|
|
|
+ ApplicationAttemptId applicationAttemptId =
|
|
|
+ ApplicationAttemptId.newInstance(applicationId, 1);
|
|
|
+ ContainerId contaierId = ContainerId.newInstance(applicationAttemptId, id);
|
|
|
+ ContainerStatus containerStatus =
|
|
|
+ BuilderUtils.newContainerStatus(contaierId, containerState,
|
|
|
+ "test_containerStatus: id=" + id + ", containerState: "
|
|
|
+ + containerState, 0);
|
|
|
+ return containerStatus;
|
|
|
+ }
|
|
|
|
|
|
- private Container getMockContainer(ContainerStatus containerStatus) {
|
|
|
- Container container = mock(Container.class);
|
|
|
- when(container.cloneAndGetContainerStatus()).thenReturn(containerStatus);
|
|
|
- return container;
|
|
|
- }
|
|
|
+ public static Container getMockContainer(ContainerStatus containerStatus) {
|
|
|
+ ContainerImpl container = mock(ContainerImpl.class);
|
|
|
+ when(container.cloneAndGetContainerStatus()).thenReturn(containerStatus);
|
|
|
+ when(container.getCurrentState()).thenReturn(containerStatus.getState());
|
|
|
+ when(container.getContainerId()).thenReturn(
|
|
|
+ containerStatus.getContainerId());
|
|
|
+ return container;
|
|
|
}
|
|
|
|
|
|
private void verifyNodeStartFailure(String errMessage) throws Exception {
|