|
@@ -18,6 +18,11 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.Arrays;
|
|
|
+import java.util.List;
|
|
|
+
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
@@ -32,42 +37,27 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
|
-import org.apache.hadoop.yarn.api.records.LocalResource;
|
|
|
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
|
|
|
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
|
|
-import org.apache.hadoop.yarn.api.records.URL;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
-
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
|
|
|
-
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
|
|
|
- .ContainersMonitorImpl;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.MockResourceCalculatorPlugin;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.MockResourceCalculatorProcessTree;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
-import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
-
|
|
|
-import java.io.File;
|
|
|
-import java.io.IOException;
|
|
|
-import java.io.PrintWriter;
|
|
|
-import java.util.ArrayList;
|
|
|
-import java.util.Arrays;
|
|
|
-import java.util.HashMap;
|
|
|
-import java.util.List;
|
|
|
-import java.util.Map;
|
|
|
-
|
|
|
-public class TestQueuingContainerManager extends TestContainerManager {
|
|
|
+/**
|
|
|
+ * Class for testing the {@link QueuingContainerManagerImpl}.
|
|
|
+ */
|
|
|
+public class TestQueuingContainerManager extends BaseContainerManagerTest {
|
|
|
|
|
|
interface HasResources {
|
|
|
boolean decide(Context context, ContainerId cId);
|
|
@@ -119,15 +109,6 @@ public class TestQueuingContainerManager extends TestContainerManager {
|
|
|
return ugi;
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- protected void authorizeGetAndStopContainerRequest(
|
|
|
- ContainerId containerId, Container container, boolean stopRequest,
|
|
|
- NMTokenIdentifier identifier) throws YarnException {
|
|
|
- if (container == null || container.getUser().equals("Fail")) {
|
|
|
- throw new YarnException("Reject this container");
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
protected ContainersMonitor createContainersMonitor(
|
|
|
ContainerExecutor exec) {
|
|
@@ -148,7 +129,7 @@ public class TestQueuingContainerManager extends TestContainerManager {
|
|
|
|
|
|
@Override
|
|
|
public long getVCoresAllocatedForContainers() {
|
|
|
- return 2;
|
|
|
+ return 4;
|
|
|
}
|
|
|
};
|
|
|
}
|
|
@@ -186,54 +167,17 @@ public class TestQueuingContainerManager extends TestContainerManager {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Test to verify that an OPPORTUNISTIC container is killed when
|
|
|
- * a GUARANTEED container arrives and all the Node Resources are used up
|
|
|
- *
|
|
|
- * For this specific test case, 4 containers are requested (last one being
|
|
|
- * guaranteed). Assumptions :
|
|
|
- * 1) The first OPPORTUNISTIC Container will start running
|
|
|
- * 2) The second and third OPP containers will be queued
|
|
|
- * 3) When the GUARANTEED container comes in, the running OPP container
|
|
|
- * will be killed to make room
|
|
|
- * 4) After the GUARANTEED container finishes, the remaining 2 OPP
|
|
|
- * containers will be dequeued and run.
|
|
|
- * 5) Only the first OPP container will be killed.
|
|
|
- *
|
|
|
+ * Starting one GUARANTEED and one OPPORTUNISTIC container.
|
|
|
* @throws Exception
|
|
|
*/
|
|
|
@Test
|
|
|
- public void testSimpleOpportunisticContainer() throws Exception {
|
|
|
+ public void testStartMultipleContainers() throws Exception {
|
|
|
shouldDeleteWait = true;
|
|
|
containerManager.start();
|
|
|
|
|
|
- // ////// Create the resources for the container
|
|
|
- File dir = new File(tmpDir, "dir");
|
|
|
- dir.mkdirs();
|
|
|
- File file = new File(dir, "file");
|
|
|
- PrintWriter fileWriter = new PrintWriter(file);
|
|
|
- fileWriter.write("Hello World!");
|
|
|
- fileWriter.close();
|
|
|
-
|
|
|
- // ////// Construct the container-spec.
|
|
|
ContainerLaunchContext containerLaunchContext =
|
|
|
recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
|
- URL resource_alpha =
|
|
|
- ConverterUtils.getYarnUrlFromPath(localFS
|
|
|
- .makeQualified(new Path(file.getAbsolutePath())));
|
|
|
- LocalResource rsrc_alpha =
|
|
|
- recordFactory.newRecordInstance(LocalResource.class);
|
|
|
- rsrc_alpha.setResource(resource_alpha);
|
|
|
- rsrc_alpha.setSize(-1);
|
|
|
- rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
|
|
|
- rsrc_alpha.setType(LocalResourceType.FILE);
|
|
|
- rsrc_alpha.setTimestamp(file.lastModified());
|
|
|
- String destinationFile = "dest_file";
|
|
|
- Map<String, LocalResource> localResources =
|
|
|
- new HashMap<String, LocalResource>();
|
|
|
- localResources.put(destinationFile, rsrc_alpha);
|
|
|
- containerLaunchContext.setLocalResources(localResources);
|
|
|
-
|
|
|
- // Start 3 OPPORTUNISTIC containers and 1 GUARANTEED container
|
|
|
+
|
|
|
List<StartContainerRequest> list = new ArrayList<>();
|
|
|
list.add(StartContainerRequest.newInstance(
|
|
|
containerLaunchContext,
|
|
@@ -241,6 +185,122 @@ public class TestQueuingContainerManager extends TestContainerManager {
|
|
|
context.getNodeId(),
|
|
|
user, BuilderUtils.newResource(1024, 1),
|
|
|
context.getContainerTokenSecretManager(), null,
|
|
|
+ ExecutionType.GUARANTEED)));
|
|
|
+ list.add(StartContainerRequest.newInstance(
|
|
|
+ containerLaunchContext,
|
|
|
+ createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
|
|
|
+ context.getNodeId(),
|
|
|
+ user, BuilderUtils.newResource(1024, 1),
|
|
|
+ context.getContainerTokenSecretManager(), null,
|
|
|
+ ExecutionType.OPPORTUNISTIC)));
|
|
|
+
|
|
|
+ StartContainersRequest allRequests =
|
|
|
+ StartContainersRequest.newInstance(list);
|
|
|
+ containerManager.startContainers(allRequests);
|
|
|
+
|
|
|
+ BaseContainerManagerTest.waitForContainerState(containerManager,
|
|
|
+ createContainerId(0),
|
|
|
+ org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
|
|
|
+ BaseContainerManagerTest.waitForContainerState(containerManager,
|
|
|
+ createContainerId(1),
|
|
|
+ org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
|
|
|
+
|
|
|
+ // Ensure all containers are running.
|
|
|
+ List<ContainerId> statList = new ArrayList<ContainerId>();
|
|
|
+ for (int i = 0; i < 2; i++) {
|
|
|
+ statList.add(createContainerId(i));
|
|
|
+ }
|
|
|
+ GetContainerStatusesRequest statRequest =
|
|
|
+ GetContainerStatusesRequest.newInstance(statList);
|
|
|
+ List<ContainerStatus> containerStatuses = containerManager
|
|
|
+ .getContainerStatuses(statRequest).getContainerStatuses();
|
|
|
+ for (ContainerStatus status : containerStatuses) {
|
|
|
+ Assert.assertEquals(
|
|
|
+ org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
|
|
|
+ status.getState());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Submit both a GUARANTEED and an OPPORTUNISTIC container, each of which
|
|
|
+ * requires more resources than available at the node, and make sure they
|
|
|
+ * are both queued.
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testQueueMultipleContainers() throws Exception {
|
|
|
+ shouldDeleteWait = true;
|
|
|
+ containerManager.start();
|
|
|
+
|
|
|
+ ContainerLaunchContext containerLaunchContext =
|
|
|
+ recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
|
+
|
|
|
+ List<StartContainerRequest> list = new ArrayList<>();
|
|
|
+ list.add(StartContainerRequest.newInstance(
|
|
|
+ containerLaunchContext,
|
|
|
+ createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
|
|
|
+ context.getNodeId(),
|
|
|
+ user, BuilderUtils.newResource(3072, 1),
|
|
|
+ context.getContainerTokenSecretManager(), null,
|
|
|
+ ExecutionType.GUARANTEED)));
|
|
|
+ list.add(StartContainerRequest.newInstance(
|
|
|
+ containerLaunchContext,
|
|
|
+ createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
|
|
|
+ context.getNodeId(),
|
|
|
+ user, BuilderUtils.newResource(3072, 1),
|
|
|
+ context.getContainerTokenSecretManager(), null,
|
|
|
+ ExecutionType.OPPORTUNISTIC)));
|
|
|
+
|
|
|
+ StartContainersRequest allRequests =
|
|
|
+ StartContainersRequest.newInstance(list);
|
|
|
+ containerManager.startContainers(allRequests);
|
|
|
+
|
|
|
+ Thread.sleep(5000);
|
|
|
+
|
|
|
+ // Ensure both containers are queued.
|
|
|
+ List<ContainerId> statList = new ArrayList<ContainerId>();
|
|
|
+ for (int i = 0; i < 2; i++) {
|
|
|
+ statList.add(createContainerId(i));
|
|
|
+ }
|
|
|
+ GetContainerStatusesRequest statRequest =
|
|
|
+ GetContainerStatusesRequest.newInstance(statList);
|
|
|
+ List<ContainerStatus> containerStatuses = containerManager
|
|
|
+ .getContainerStatuses(statRequest).getContainerStatuses();
|
|
|
+ for (ContainerStatus status : containerStatuses) {
|
|
|
+ Assert.assertEquals(
|
|
|
+ org.apache.hadoop.yarn.api.records.ContainerState.QUEUED,
|
|
|
+ status.getState());
|
|
|
+ }
|
|
|
+
|
|
|
+ // Ensure both containers are properly queued.
|
|
|
+ Assert.assertEquals(2, containerManager.getContext().getQueuingContext()
|
|
|
+ .getQueuedContainers().size());
|
|
|
+ Assert.assertEquals(1, ((QueuingContainerManagerImpl) containerManager)
|
|
|
+ .getNumQueuedGuaranteedContainers());
|
|
|
+ Assert.assertEquals(1, ((QueuingContainerManagerImpl) containerManager)
|
|
|
+ .getNumQueuedOpportunisticContainers());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Starts one OPPORTUNISTIC container that takes up the whole node's
|
|
|
+ * resources, and submit two more that will be queued.
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testStartAndQueueMultipleContainers() throws Exception {
|
|
|
+ shouldDeleteWait = true;
|
|
|
+ containerManager.start();
|
|
|
+
|
|
|
+ ContainerLaunchContext containerLaunchContext =
|
|
|
+ recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
|
+
|
|
|
+ List<StartContainerRequest> list = new ArrayList<>();
|
|
|
+ list.add(StartContainerRequest.newInstance(
|
|
|
+ containerLaunchContext,
|
|
|
+ createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
|
|
|
+ context.getNodeId(),
|
|
|
+ user, BuilderUtils.newResource(2048, 1),
|
|
|
+ context.getContainerTokenSecretManager(), null,
|
|
|
ExecutionType.OPPORTUNISTIC)));
|
|
|
list.add(StartContainerRequest.newInstance(
|
|
|
containerLaunchContext,
|
|
@@ -256,23 +316,95 @@ public class TestQueuingContainerManager extends TestContainerManager {
|
|
|
user, BuilderUtils.newResource(1024, 1),
|
|
|
context.getContainerTokenSecretManager(), null,
|
|
|
ExecutionType.OPPORTUNISTIC)));
|
|
|
- // GUARANTEED
|
|
|
+
|
|
|
+ StartContainersRequest allRequests =
|
|
|
+ StartContainersRequest.newInstance(list);
|
|
|
+ containerManager.startContainers(allRequests);
|
|
|
+
|
|
|
+ Thread.sleep(5000);
|
|
|
+
|
|
|
+ // Ensure first container is running and others are queued.
|
|
|
+ List<ContainerId> statList = new ArrayList<ContainerId>();
|
|
|
+ for (int i = 0; i < 3; i++) {
|
|
|
+ statList.add(createContainerId(i));
|
|
|
+ }
|
|
|
+ GetContainerStatusesRequest statRequest = GetContainerStatusesRequest
|
|
|
+ .newInstance(Arrays.asList(createContainerId(0)));
|
|
|
+ List<ContainerStatus> containerStatuses = containerManager
|
|
|
+ .getContainerStatuses(statRequest).getContainerStatuses();
|
|
|
+ for (ContainerStatus status : containerStatuses) {
|
|
|
+ if (status.getContainerId().equals(createContainerId(0))) {
|
|
|
+ Assert.assertEquals(
|
|
|
+ org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
|
|
|
+ status.getState());
|
|
|
+ } else {
|
|
|
+ Assert.assertEquals(
|
|
|
+ org.apache.hadoop.yarn.api.records.ContainerState.QUEUED,
|
|
|
+ status.getState());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Ensure two containers are properly queued.
|
|
|
+ Assert.assertEquals(2, containerManager.getContext().getQueuingContext()
|
|
|
+ .getQueuedContainers().size());
|
|
|
+ Assert.assertEquals(0, ((QueuingContainerManagerImpl) containerManager)
|
|
|
+ .getNumQueuedGuaranteedContainers());
|
|
|
+ Assert.assertEquals(2, ((QueuingContainerManagerImpl) containerManager)
|
|
|
+ .getNumQueuedOpportunisticContainers());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Submit two OPPORTUNISTIC and one GUARANTEED containers. The resources
|
|
|
+ * requests by each container as such that only one can run in parallel.
|
|
|
+ * Thus, the OPPORTUNISTIC container that started running, will be
|
|
|
+ * killed for the GUARANTEED container to start.
|
|
|
+ * Once the GUARANTEED container finishes its execution, the remaining
|
|
|
+ * OPPORTUNISTIC container will be executed.
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testKillOpportunisticForGuaranteedContainer() throws Exception {
|
|
|
+ shouldDeleteWait = true;
|
|
|
+ containerManager.start();
|
|
|
+
|
|
|
+ ContainerLaunchContext containerLaunchContext =
|
|
|
+ recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
|
+
|
|
|
+ List<StartContainerRequest> list = new ArrayList<>();
|
|
|
+ list.add(StartContainerRequest.newInstance(
|
|
|
+ containerLaunchContext,
|
|
|
+ createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
|
|
|
+ context.getNodeId(),
|
|
|
+ user, BuilderUtils.newResource(2048, 1),
|
|
|
+ context.getContainerTokenSecretManager(), null,
|
|
|
+ ExecutionType.OPPORTUNISTIC)));
|
|
|
list.add(StartContainerRequest.newInstance(
|
|
|
containerLaunchContext,
|
|
|
- createContainerToken(createContainerId(3), DUMMY_RM_IDENTIFIER,
|
|
|
+ createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
|
|
|
context.getNodeId(),
|
|
|
- user, BuilderUtils.newResource(1024, 1),
|
|
|
+ user, BuilderUtils.newResource(2048, 1),
|
|
|
+ context.getContainerTokenSecretManager(), null,
|
|
|
+ ExecutionType.OPPORTUNISTIC)));
|
|
|
+ list.add(StartContainerRequest.newInstance(
|
|
|
+ containerLaunchContext,
|
|
|
+ createContainerToken(createContainerId(2), DUMMY_RM_IDENTIFIER,
|
|
|
+ context.getNodeId(),
|
|
|
+ user, BuilderUtils.newResource(2048, 1),
|
|
|
context.getContainerTokenSecretManager(), null,
|
|
|
ExecutionType.GUARANTEED)));
|
|
|
+
|
|
|
StartContainersRequest allRequests =
|
|
|
StartContainersRequest.newInstance(list);
|
|
|
-
|
|
|
containerManager.startContainers(allRequests);
|
|
|
|
|
|
- Thread.sleep(10000);
|
|
|
+ BaseContainerManagerTest.waitForNMContainerState(containerManager,
|
|
|
+ createContainerId(0), ContainerState.DONE, 30);
|
|
|
+ Thread.sleep(5000);
|
|
|
|
|
|
+ // Get container statuses. Container 0 should be killed, container 1
|
|
|
+ // should be queued and container 2 should be running.
|
|
|
List<ContainerId> statList = new ArrayList<ContainerId>();
|
|
|
- for (int i = 0; i < 4; i++) {
|
|
|
+ for (int i = 0; i < 3; i++) {
|
|
|
statList.add(createContainerId(i));
|
|
|
}
|
|
|
GetContainerStatusesRequest statRequest =
|
|
@@ -280,12 +412,110 @@ public class TestQueuingContainerManager extends TestContainerManager {
|
|
|
List<ContainerStatus> containerStatuses = containerManager
|
|
|
.getContainerStatuses(statRequest).getContainerStatuses();
|
|
|
for (ContainerStatus status : containerStatuses) {
|
|
|
- // Ensure that the first opportunistic container is killed
|
|
|
if (status.getContainerId().equals(createContainerId(0))) {
|
|
|
Assert.assertTrue(status.getDiagnostics()
|
|
|
.contains("Container killed by the ApplicationMaster"));
|
|
|
+ } else if (status.getContainerId().equals(createContainerId(1))) {
|
|
|
+ Assert.assertEquals(
|
|
|
+ org.apache.hadoop.yarn.api.records.ContainerState.QUEUED,
|
|
|
+ status.getState());
|
|
|
+ } else if (status.getContainerId().equals(createContainerId(2))) {
|
|
|
+ Assert.assertEquals(
|
|
|
+ org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
|
|
|
+ status.getState());
|
|
|
+ }
|
|
|
+ System.out.println("\nStatus : [" + status + "]\n");
|
|
|
+ }
|
|
|
+
|
|
|
+ // Make sure the remaining OPPORTUNISTIC container starts its execution.
|
|
|
+ BaseContainerManagerTest.waitForNMContainerState(containerManager,
|
|
|
+ createContainerId(2), ContainerState.DONE, 30);
|
|
|
+ Thread.sleep(5000);
|
|
|
+ statRequest = GetContainerStatusesRequest.newInstance(Arrays.asList(
|
|
|
+ createContainerId(1)));
|
|
|
+ ContainerStatus contStatus1 = containerManager.getContainerStatuses(
|
|
|
+ statRequest).getContainerStatuses().get(0);
|
|
|
+ Assert.assertEquals(
|
|
|
+ org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
|
|
|
+ contStatus1.getState());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Submit three OPPORTUNISTIC containers that can run concurrently, and one
|
|
|
+ * GUARANTEED that needs to kill two of the OPPORTUNISTIC for it to run.
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testKillMultipleOpportunisticContainers() throws Exception {
|
|
|
+ shouldDeleteWait = true;
|
|
|
+ containerManager.start();
|
|
|
+
|
|
|
+ ContainerLaunchContext containerLaunchContext =
|
|
|
+ recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
|
+
|
|
|
+ List<StartContainerRequest> list = new ArrayList<>();
|
|
|
+ list.add(StartContainerRequest.newInstance(
|
|
|
+ containerLaunchContext,
|
|
|
+ createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
|
|
|
+ context.getNodeId(),
|
|
|
+ user, BuilderUtils.newResource(512, 1),
|
|
|
+ context.getContainerTokenSecretManager(), null,
|
|
|
+ ExecutionType.OPPORTUNISTIC)));
|
|
|
+ list.add(StartContainerRequest.newInstance(
|
|
|
+ containerLaunchContext,
|
|
|
+ createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
|
|
|
+ context.getNodeId(),
|
|
|
+ user, BuilderUtils.newResource(512, 1),
|
|
|
+ context.getContainerTokenSecretManager(), null,
|
|
|
+ ExecutionType.OPPORTUNISTIC)));
|
|
|
+ list.add(StartContainerRequest.newInstance(
|
|
|
+ containerLaunchContext,
|
|
|
+ createContainerToken(createContainerId(2), DUMMY_RM_IDENTIFIER,
|
|
|
+ context.getNodeId(),
|
|
|
+ user, BuilderUtils.newResource(512, 1),
|
|
|
+ context.getContainerTokenSecretManager(), null,
|
|
|
+ ExecutionType.OPPORTUNISTIC)));
|
|
|
+ list.add(StartContainerRequest.newInstance(
|
|
|
+ containerLaunchContext,
|
|
|
+ createContainerToken(createContainerId(3), DUMMY_RM_IDENTIFIER,
|
|
|
+ context.getNodeId(),
|
|
|
+ user, BuilderUtils.newResource(1500, 1),
|
|
|
+ context.getContainerTokenSecretManager(), null,
|
|
|
+ ExecutionType.GUARANTEED)));
|
|
|
+
|
|
|
+ StartContainersRequest allRequests =
|
|
|
+ StartContainersRequest.newInstance(list);
|
|
|
+ containerManager.startContainers(allRequests);
|
|
|
+
|
|
|
+ BaseContainerManagerTest.waitForNMContainerState(containerManager,
|
|
|
+ createContainerId(0), ContainerState.DONE, 30);
|
|
|
+ Thread.sleep(5000);
|
|
|
+
|
|
|
+ // Get container statuses. Container 0 should be killed, container 1
|
|
|
+ // should be queued and container 2 should be running.
|
|
|
+ int killedContainers = 0;
|
|
|
+ int runningContainers = 0;
|
|
|
+ List<ContainerId> statList = new ArrayList<ContainerId>();
|
|
|
+ for (int i = 0; i < 4; i++) {
|
|
|
+ statList.add(createContainerId(i));
|
|
|
+ }
|
|
|
+ GetContainerStatusesRequest statRequest =
|
|
|
+ GetContainerStatusesRequest.newInstance(statList);
|
|
|
+ List<ContainerStatus> containerStatuses = containerManager
|
|
|
+ .getContainerStatuses(statRequest).getContainerStatuses();
|
|
|
+ for (ContainerStatus status : containerStatuses) {
|
|
|
+ if (status.getDiagnostics().contains(
|
|
|
+ "Container killed by the ApplicationMaster")) {
|
|
|
+ killedContainers++;
|
|
|
+ }
|
|
|
+ if (status.getState() ==
|
|
|
+ org.apache.hadoop.yarn.api.records.ContainerState.RUNNING) {
|
|
|
+ runningContainers++;
|
|
|
}
|
|
|
System.out.println("\nStatus : [" + status + "]\n");
|
|
|
}
|
|
|
+
|
|
|
+ Assert.assertEquals(2, killedContainers);
|
|
|
+ Assert.assertEquals(2, runningContainers);
|
|
|
}
|
|
|
}
|