|
@@ -25,6 +25,7 @@ import static org.mockito.Matchers.any;
|
|
|
import static org.mockito.Matchers.argThat;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.reset;
|
|
|
+import static org.mockito.Mockito.times;
|
|
|
import static org.mockito.Mockito.verify;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
import static org.mockito.Mockito.atLeastOnce;
|
|
@@ -73,12 +74,14 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerStateTransitionListener;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.deletion.task.DockerContainerDeletionMatcher;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncher;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEvent;
|
|
@@ -94,7 +97,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.Contai
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEventType;
|
|
|
|
|
|
-
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeConstants;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerScheduler;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerSchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerSchedulerEventType;
|
|
@@ -208,6 +211,37 @@ public class TestContainer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ @SuppressWarnings("unchecked") // mocked generic
|
|
|
+ public void testDockerContainerExternalKill() throws Exception {
|
|
|
+ WrappedContainer wc = null;
|
|
|
+ try {
|
|
|
+ wc = new WrappedContainer(13, 314159265358979L, 4344, "yak");
|
|
|
+ wc.setupDockerContainerEnv();
|
|
|
+ wc.initContainer();
|
|
|
+ wc.localizeResources();
|
|
|
+ int running = metrics.getRunningContainers();
|
|
|
+ wc.launchContainer();
|
|
|
+ assertEquals(running + 1, metrics.getRunningContainers());
|
|
|
+ reset(wc.localizerBus);
|
|
|
+ wc.containerKilledOnRequest();
|
|
|
+ assertEquals(ContainerState.EXITED_WITH_FAILURE,
|
|
|
+ wc.c.getContainerState());
|
|
|
+ assertNull(wc.c.getLocalizedResources());
|
|
|
+ verifyCleanupCall(wc);
|
|
|
+ int failed = metrics.getFailedContainers();
|
|
|
+ wc.dockerContainerResourcesCleanup();
|
|
|
+ assertEquals(ContainerState.DONE, wc.c.getContainerState());
|
|
|
+ assertEquals(failed + 1, metrics.getFailedContainers());
|
|
|
+ assertEquals(running, metrics.getRunningContainers());
|
|
|
+ }
|
|
|
+ finally {
|
|
|
+ if (wc != null) {
|
|
|
+ wc.finished();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
@SuppressWarnings("unchecked") // mocked generic
|
|
|
public void testContainerPauseAndResume() throws Exception {
|
|
@@ -266,6 +300,30 @@ public class TestContainer {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ @SuppressWarnings("unchecked") // mocked generic
|
|
|
+ public void testDockerContainerCleanupOnFailure() throws Exception {
|
|
|
+ WrappedContainer wc = null;
|
|
|
+ try {
|
|
|
+ wc = new WrappedContainer(10, 314159265358979L, 4344, "yak");
|
|
|
+ wc.setupDockerContainerEnv();
|
|
|
+ wc.initContainer();
|
|
|
+ wc.localizeResources();
|
|
|
+ wc.launchContainer();
|
|
|
+ reset(wc.localizerBus);
|
|
|
+ wc.containerFailed(ExitCode.FORCE_KILLED.getExitCode());
|
|
|
+ assertEquals(ContainerState.EXITED_WITH_FAILURE,
|
|
|
+ wc.c.getContainerState());
|
|
|
+ assertNull(wc.c.getLocalizedResources());
|
|
|
+ verifyCleanupCall(wc);
|
|
|
+ wc.dockerContainerResourcesCleanup();
|
|
|
+ } finally {
|
|
|
+ if (wc != null) {
|
|
|
+ wc.finished();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
@Test
|
|
|
@SuppressWarnings("unchecked") // mocked generic
|
|
@@ -320,6 +378,36 @@ public class TestContainer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ @SuppressWarnings("unchecked") // mocked generic
|
|
|
+ public void testDockerContainerCleanupOnSuccess() throws Exception {
|
|
|
+ WrappedContainer wc = null;
|
|
|
+ try {
|
|
|
+ wc = new WrappedContainer(11, 314159265358979L, 4344, "yak");
|
|
|
+ wc.setupDockerContainerEnv();
|
|
|
+ wc.initContainer();
|
|
|
+ wc.localizeResources();
|
|
|
+ int running = metrics.getRunningContainers();
|
|
|
+ wc.launchContainer();
|
|
|
+ assertEquals(running + 1, metrics.getRunningContainers());
|
|
|
+ reset(wc.localizerBus);
|
|
|
+ wc.containerSuccessful();
|
|
|
+ assertEquals(ContainerState.EXITED_WITH_SUCCESS,
|
|
|
+ wc.c.getContainerState());
|
|
|
+ assertNull(wc.c.getLocalizedResources());
|
|
|
+ verifyCleanupCall(wc);
|
|
|
+ int completed = metrics.getCompletedContainers();
|
|
|
+ wc.dockerContainerResourcesCleanup();
|
|
|
+ assertEquals(ContainerState.DONE, wc.c.getContainerState());
|
|
|
+ assertEquals(completed + 1, metrics.getCompletedContainers());
|
|
|
+ assertEquals(running, metrics.getRunningContainers());
|
|
|
+ } finally {
|
|
|
+ if (wc != null) {
|
|
|
+ wc.finished();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
@SuppressWarnings("unchecked") // mocked generic
|
|
|
public void testInitWhileDone() throws Exception {
|
|
@@ -341,8 +429,36 @@ public class TestContainer {
|
|
|
assertEquals(ContainerState.DONE, wc.c.getContainerState());
|
|
|
assertNull(wc.c.getLocalizedResources());
|
|
|
verifyCleanupCall(wc);
|
|
|
+ } finally {
|
|
|
+ if (wc != null) {
|
|
|
+ wc.finished();
|
|
|
+ }
|
|
|
}
|
|
|
- finally {
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ @SuppressWarnings("unchecked") // mocked generic
|
|
|
+ public void testDockerContainerInitWhileDone() throws Exception {
|
|
|
+ WrappedContainer wc = null;
|
|
|
+ try {
|
|
|
+ wc = new WrappedContainer(6, 314159265358979L, 4344, "yak");
|
|
|
+ wc.setupDockerContainerEnv();
|
|
|
+ wc.initContainer();
|
|
|
+ wc.localizeResources();
|
|
|
+ wc.launchContainer();
|
|
|
+ reset(wc.localizerBus);
|
|
|
+ wc.containerSuccessful();
|
|
|
+ wc.dockerContainerResourcesCleanup();
|
|
|
+ assertEquals(ContainerState.DONE, wc.c.getContainerState());
|
|
|
+ verifyOutofBandHeartBeat(wc);
|
|
|
+ assertNull(wc.c.getLocalizedResources());
|
|
|
+ // Now in DONE, issue INIT
|
|
|
+ wc.initContainer();
|
|
|
+ // Verify still in DONE
|
|
|
+ assertEquals(ContainerState.DONE, wc.c.getContainerState());
|
|
|
+ assertNull(wc.c.getLocalizedResources());
|
|
|
+ verifyCleanupCall(wc);
|
|
|
+ } finally {
|
|
|
if (wc != null) {
|
|
|
wc.finished();
|
|
|
}
|
|
@@ -377,6 +493,36 @@ public class TestContainer {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ // mocked generic
|
|
|
+ public void testDockerContainerLocalizationFailureAtDone() throws Exception {
|
|
|
+ WrappedContainer wc = null;
|
|
|
+ try {
|
|
|
+ wc = new WrappedContainer(6, 314159265358979L, 4344, "yak");
|
|
|
+ wc.setupDockerContainerEnv();
|
|
|
+ wc.initContainer();
|
|
|
+ wc.localizeResources();
|
|
|
+ wc.launchContainer();
|
|
|
+ reset(wc.localizerBus);
|
|
|
+ wc.containerSuccessful();
|
|
|
+ wc.dockerContainerResourcesCleanup();
|
|
|
+ assertEquals(ContainerState.DONE, wc.c.getContainerState());
|
|
|
+ verifyOutofBandHeartBeat(wc);
|
|
|
+ assertNull(wc.c.getLocalizedResources());
|
|
|
+ // Now in DONE, issue RESOURCE_FAILED as done by LocalizeRunner
|
|
|
+ wc.resourceFailedContainer();
|
|
|
+ // Verify still in DONE
|
|
|
+ assertEquals(ContainerState.DONE, wc.c.getContainerState());
|
|
|
+ assertNull(wc.c.getLocalizedResources());
|
|
|
+ verifyCleanupCall(wc);
|
|
|
+ } finally {
|
|
|
+ if (wc != null) {
|
|
|
+ wc.finished();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
@Test
|
|
|
@SuppressWarnings("unchecked") // mocked generic
|
|
@@ -516,6 +662,38 @@ public class TestContainer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testDockerKillOnLocalizedWhenContainerNotLaunchedContainerKilled()
|
|
|
+ throws Exception {
|
|
|
+ WrappedContainer wc = null;
|
|
|
+ try {
|
|
|
+ wc = new WrappedContainer(17, 314159265358979L, 4344, "yak");
|
|
|
+ wc.setupDockerContainerEnv();
|
|
|
+ wc.initContainer();
|
|
|
+ wc.localizeResources();
|
|
|
+ assertEquals(ContainerState.SCHEDULED, wc.c.getContainerState());
|
|
|
+ ContainerLaunch launcher = wc.launcher.running.get(wc.c.getContainerId());
|
|
|
+ wc.killContainer();
|
|
|
+ assertEquals(ContainerState.KILLING, wc.c.getContainerState());
|
|
|
+ launcher.call();
|
|
|
+ wc.drainDispatcherEvents();
|
|
|
+ assertEquals(ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
|
|
|
+ wc.c.getContainerState());
|
|
|
+ assertNull(wc.c.getLocalizedResources());
|
|
|
+ verifyDockerContainerCleanupCall(wc);
|
|
|
+ int killed = metrics.getKilledContainers();
|
|
|
+ wc.c.handle(new ContainerEvent(wc.c.getContainerId(),
|
|
|
+ ContainerEventType.CONTAINER_RESOURCES_CLEANEDUP));
|
|
|
+ assertEquals(ContainerState.DONE, wc.c.getContainerState());
|
|
|
+ assertEquals(killed + 1, metrics.getKilledContainers());
|
|
|
+ assertEquals(0, metrics.getRunningContainers());
|
|
|
+ } finally {
|
|
|
+ if (wc != null) {
|
|
|
+ wc.finished();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testKillOnLocalizedWhenContainerNotLaunchedContainerSuccess()
|
|
|
throws Exception {
|
|
@@ -572,6 +750,35 @@ public class TestContainer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testDockerKillOnLocalizedContainerNotLaunchedContainerFailure()
|
|
|
+ throws Exception {
|
|
|
+ WrappedContainer wc = null;
|
|
|
+ try {
|
|
|
+ wc = new WrappedContainer(17, 314159265358979L, 4344, "yak");
|
|
|
+ wc.setupDockerContainerEnv();
|
|
|
+ wc.initContainer();
|
|
|
+ wc.localizeResources();
|
|
|
+ assertEquals(ContainerState.SCHEDULED, wc.c.getContainerState());
|
|
|
+ wc.killContainer();
|
|
|
+ assertEquals(ContainerState.KILLING, wc.c.getContainerState());
|
|
|
+ wc.containerFailed(ExitCode.FORCE_KILLED.getExitCode());
|
|
|
+ wc.drainDispatcherEvents();
|
|
|
+ assertEquals(ContainerState.EXITED_WITH_FAILURE,
|
|
|
+ wc.c.getContainerState());
|
|
|
+ assertNull(wc.c.getLocalizedResources());
|
|
|
+ verifyDockerContainerCleanupCall(wc);
|
|
|
+ wc.c.handle(new ContainerEvent(wc.c.getContainerId(),
|
|
|
+ ContainerEventType.CONTAINER_RESOURCES_CLEANEDUP));
|
|
|
+ assertEquals(ContainerState.DONE, wc.c.getContainerState());
|
|
|
+ assertEquals(0, metrics.getRunningContainers());
|
|
|
+ } finally {
|
|
|
+ if (wc != null) {
|
|
|
+ wc.finished();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testKillOnLocalizedWhenContainerLaunched() throws Exception {
|
|
|
WrappedContainer wc = null;
|
|
@@ -596,6 +803,33 @@ public class TestContainer {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testDockerKillOnLocalizedWhenContainerLaunched()
|
|
|
+ throws Exception {
|
|
|
+ WrappedContainer wc = null;
|
|
|
+ try {
|
|
|
+ wc = new WrappedContainer(17, 314159265358979L, 4344, "yak");
|
|
|
+ wc.setupDockerContainerEnv();
|
|
|
+ wc.initContainer();
|
|
|
+ wc.localizeResources();
|
|
|
+ assertEquals(ContainerState.SCHEDULED, wc.c.getContainerState());
|
|
|
+ ContainerLaunch launcher = wc.launcher.running.get(wc.c.getContainerId());
|
|
|
+ launcher.call();
|
|
|
+ wc.drainDispatcherEvents();
|
|
|
+ assertEquals(ContainerState.EXITED_WITH_FAILURE,
|
|
|
+ wc.c.getContainerState());
|
|
|
+ wc.killContainer();
|
|
|
+ assertEquals(ContainerState.EXITED_WITH_FAILURE,
|
|
|
+ wc.c.getContainerState());
|
|
|
+ assertNull(wc.c.getLocalizedResources());
|
|
|
+ verifyDockerContainerCleanupCall(wc);
|
|
|
+ } finally {
|
|
|
+ if (wc != null) {
|
|
|
+ wc.finished();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
@Test
|
|
|
public void testResourceLocalizedOnLocalizationFailed() throws Exception {
|
|
@@ -733,6 +967,29 @@ public class TestContainer {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testDockerContainerLaunchAfterKillRequest() throws Exception {
|
|
|
+ WrappedContainer wc = null;
|
|
|
+ try {
|
|
|
+ wc = new WrappedContainer(14, 314159265358979L, 4344, "yak");
|
|
|
+ wc.setupDockerContainerEnv();
|
|
|
+ wc.initContainer();
|
|
|
+ wc.localizeResources();
|
|
|
+ wc.killContainer();
|
|
|
+ assertEquals(ContainerState.KILLING, wc.c.getContainerState());
|
|
|
+ assertNull(wc.c.getLocalizedResources());
|
|
|
+ wc.launchContainer();
|
|
|
+ assertEquals(ContainerState.KILLING, wc.c.getContainerState());
|
|
|
+ assertNull(wc.c.getLocalizedResources());
|
|
|
+ wc.containerKilledOnRequest();
|
|
|
+ verifyDockerContainerCleanupCall(wc);
|
|
|
+ } finally {
|
|
|
+ if (wc != null) {
|
|
|
+ wc.finished();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
@Test
|
|
|
public void testContainerRetry() throws Exception{
|
|
@@ -843,6 +1100,14 @@ public class TestContainer {
|
|
|
verify(wc.context.getNodeStatusUpdater()).sendOutofBandHeartBeat();
|
|
|
}
|
|
|
|
|
|
+ private void verifyDockerContainerCleanupCall(WrappedContainer wc)
|
|
|
+ throws Exception {
|
|
|
+ DeletionService delService = wc.context.getDeletionService();
|
|
|
+ verify(delService, times(1)).delete(argThat(
|
|
|
+ new DockerContainerDeletionMatcher(delService,
|
|
|
+ wc.c.getContainerId().toString())));
|
|
|
+ }
|
|
|
+
|
|
|
private static class ResourcesReleasedMatcher extends
|
|
|
ArgumentMatcher<LocalizationEvent> {
|
|
|
final HashSet<LocalResourceRequest> resources =
|
|
@@ -971,6 +1236,7 @@ public class TestContainer {
|
|
|
final Map<String, LocalResource> localResources;
|
|
|
final Map<String, ByteBuffer> serviceData;
|
|
|
final Context context = mock(Context.class);
|
|
|
+ private final DeletionService delService;
|
|
|
private final Map<ContainerState, ContainerEventType> initStateToEvent =
|
|
|
new HashMap<>();
|
|
|
private final Map<ContainerEventType, ContainerState> eventToFinalState =
|
|
@@ -1004,6 +1270,7 @@ public class TestContainer {
|
|
|
auxBus = mock(EventHandler.class);
|
|
|
appBus = mock(EventHandler.class);
|
|
|
LogBus = mock(EventHandler.class);
|
|
|
+ delService = mock(DeletionService.class);
|
|
|
schedBus = new ContainerScheduler(context, dispatcher, metrics, 0) {
|
|
|
@Override
|
|
|
protected void scheduleContainer(Container container) {
|
|
@@ -1081,6 +1348,7 @@ public class TestContainer {
|
|
|
}
|
|
|
when(ctxt.getServiceData()).thenReturn(serviceData);
|
|
|
when(ctxt.getContainerRetryContext()).thenReturn(containerRetryContext);
|
|
|
+ when(context.getDeletionService()).thenReturn(delService);
|
|
|
ContainerStateTransitionListener listener =
|
|
|
new ContainerStateTransitionListener() {
|
|
|
@Override
|
|
@@ -1213,6 +1481,20 @@ public class TestContainer {
|
|
|
drainDispatcherEvents();
|
|
|
}
|
|
|
|
|
|
+ public void dockerContainerResourcesCleanup() {
|
|
|
+ c.handle(new ContainerEvent(cId,
|
|
|
+ ContainerEventType.CONTAINER_RESOURCES_CLEANEDUP));
|
|
|
+ verify(delService, times(1)).delete(argThat(
|
|
|
+ new DockerContainerDeletionMatcher(delService, cId.toString())));
|
|
|
+ drainDispatcherEvents();
|
|
|
+ }
|
|
|
+
|
|
|
+ public void setupDockerContainerEnv() {
|
|
|
+ Map<String, String> env = new HashMap<>();
|
|
|
+ env.put(ContainerRuntimeConstants.ENV_CONTAINER_TYPE, "docker");
|
|
|
+ when(this.ctxt.getEnvironment()).thenReturn(env);
|
|
|
+ }
|
|
|
+
|
|
|
public void containerFailed(int exitCode) {
|
|
|
String diagnosticMsg = "Container completed with exit code " + exitCode;
|
|
|
c.handle(new ContainerExitEvent(cId,
|