|
@@ -43,11 +43,13 @@ import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
|
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
|
|
|
import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocolPB;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
|
|
|
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
|
|
@@ -122,6 +124,21 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
|
rm.start();
|
|
|
}
|
|
|
|
|
|
+ public void createAndStartRMWithAutoUpdateContainer() {
|
|
|
+ CapacitySchedulerConfiguration csConf =
|
|
|
+ new CapacitySchedulerConfiguration();
|
|
|
+ YarnConfiguration conf = new YarnConfiguration(csConf);
|
|
|
+ conf.setBoolean(YarnConfiguration.RM_AUTO_UPDATE_CONTAINERS, true);
|
|
|
+ conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
|
|
|
+ ResourceScheduler.class);
|
|
|
+ conf.setBoolean(
|
|
|
+ YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
|
|
|
+ conf.setInt(
|
|
|
+ YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS, 100);
|
|
|
+ rm = new MockRM(conf);
|
|
|
+ rm.start();
|
|
|
+ }
|
|
|
+
|
|
|
@After
|
|
|
public void stopRM() {
|
|
|
if (rm != null) {
|
|
@@ -548,6 +565,157 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
|
verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 600000)
|
|
|
+ public void testContainerAutoUpdateContainer() throws Exception {
|
|
|
+ rm.stop();
|
|
|
+ createAndStartRMWithAutoUpdateContainer();
|
|
|
+ MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nm1.registerNode();
|
|
|
+
|
|
|
+ OpportunisticContainerAllocatorAMService amservice =
|
|
|
+ (OpportunisticContainerAllocatorAMService) rm
|
|
|
+ .getApplicationMasterService();
|
|
|
+ RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
|
|
|
+ ApplicationAttemptId attemptId =
|
|
|
+ app1.getCurrentAppAttempt().getAppAttemptId();
|
|
|
+ MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
|
|
|
+ ResourceScheduler scheduler = rm.getResourceScheduler();
|
|
|
+ RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
|
|
|
+
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
+
|
|
|
+ ((RMNodeImpl) rmNode1)
|
|
|
+ .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
|
|
+
|
|
|
+ OpportunisticContainerContext ctxt =
|
|
|
+ ((CapacityScheduler) scheduler).getApplicationAttempt(attemptId)
|
|
|
+ .getOpportunisticContainerContext();
|
|
|
+ // Send add and update node events to AM Service.
|
|
|
+ amservice.handle(new NodeAddedSchedulerEvent(rmNode1));
|
|
|
+ amservice.handle(new NodeUpdateSchedulerEvent(rmNode1));
|
|
|
+
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
+ Thread.sleep(1000);
|
|
|
+
|
|
|
+ AllocateResponse allocateResponse = am1.allocate(Arrays.asList(
|
|
|
+ ResourceRequest.newInstance(Priority.newInstance(1), "*",
|
|
|
+ Resources.createResource(1 * GB), 2, true, null,
|
|
|
+ ExecutionTypeRequest
|
|
|
+ .newInstance(ExecutionType.OPPORTUNISTIC, true))), null);
|
|
|
+ List<Container> allocatedContainers =
|
|
|
+ allocateResponse.getAllocatedContainers();
|
|
|
+ Assert.assertEquals(2, allocatedContainers.size());
|
|
|
+ Container container = allocatedContainers.get(0);
|
|
|
+ // Start Container in NM
|
|
|
+ nm1.nodeHeartbeat(Arrays.asList(ContainerStatus
|
|
|
+ .newInstance(container.getId(), ExecutionType.OPPORTUNISTIC,
|
|
|
+ ContainerState.RUNNING, "", 0)), true);
|
|
|
+ Thread.sleep(200);
|
|
|
+
|
|
|
+ // Verify that container is actually running wrt the RM..
|
|
|
+ RMContainer rmContainer = ((CapacityScheduler) scheduler)
|
|
|
+ .getApplicationAttempt(container.getId().getApplicationAttemptId())
|
|
|
+ .getRMContainer(container.getId());
|
|
|
+ Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
|
|
|
+
|
|
|
+ // Send Promotion req... this should result in update error
|
|
|
+ // Since the container doesn't exist anymore..
|
|
|
+ allocateResponse = am1.sendContainerUpdateRequest(Arrays.asList(
|
|
|
+ UpdateContainerRequest.newInstance(0, container.getId(),
|
|
|
+ ContainerUpdateType.PROMOTE_EXECUTION_TYPE, null,
|
|
|
+ ExecutionType.GUARANTEED)));
|
|
|
+
|
|
|
+ nm1.nodeHeartbeat(Arrays.asList(ContainerStatus
|
|
|
+ .newInstance(container.getId(), ExecutionType.OPPORTUNISTIC,
|
|
|
+ ContainerState.RUNNING, "", 0)), true);
|
|
|
+ Thread.sleep(200);
|
|
|
+ // Get the update response on next allocate
|
|
|
+ allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
|
|
|
+ // Check the update response from YARNRM
|
|
|
+ Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
|
|
|
+ UpdatedContainer uc = allocateResponse.getUpdatedContainers().get(0);
|
|
|
+ Assert.assertEquals(container.getId(), uc.getContainer().getId());
|
|
|
+ Assert.assertEquals(ExecutionType.GUARANTEED,
|
|
|
+ uc.getContainer().getExecutionType());
|
|
|
+ // Check that the container is updated in NM through NM heartbeat response
|
|
|
+ NodeHeartbeatResponse response = nm1.nodeHeartbeat(true);
|
|
|
+ Assert.assertEquals(1, response.getContainersToUpdate().size());
|
|
|
+ Container containersFromNM = response.getContainersToUpdate().get(0);
|
|
|
+ Assert.assertEquals(container.getId(), containersFromNM.getId());
|
|
|
+ Assert.assertEquals(ExecutionType.GUARANTEED,
|
|
|
+ containersFromNM.getExecutionType());
|
|
|
+
|
|
|
+ //Increase resources
|
|
|
+ allocateResponse = am1.sendContainerUpdateRequest(Arrays.asList(
|
|
|
+ UpdateContainerRequest.newInstance(1, container.getId(),
|
|
|
+ ContainerUpdateType.INCREASE_RESOURCE,
|
|
|
+ Resources.createResource(2 * GB, 1), null)));
|
|
|
+ response = nm1.nodeHeartbeat(Arrays.asList(ContainerStatus
|
|
|
+ .newInstance(container.getId(), ExecutionType.GUARANTEED,
|
|
|
+ ContainerState.RUNNING, "", 0)), true);
|
|
|
+
|
|
|
+ Thread.sleep(200);
|
|
|
+ if (allocateResponse.getUpdatedContainers().size() == 0) {
|
|
|
+ allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
|
|
|
+ }
|
|
|
+ Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
|
|
|
+ uc = allocateResponse.getUpdatedContainers().get(0);
|
|
|
+ Assert.assertEquals(container.getId(), uc.getContainer().getId());
|
|
|
+ Assert.assertEquals(Resource.newInstance(2 * GB, 1),
|
|
|
+ uc.getContainer().getResource());
|
|
|
+
|
|
|
+ // Check that the container resources are increased in
|
|
|
+ // NM through NM heartbeat response
|
|
|
+ if (response.getContainersToUpdate().size() == 0) {
|
|
|
+ response = nm1.nodeHeartbeat(true);
|
|
|
+ }
|
|
|
+ Assert.assertEquals(1, response.getContainersToUpdate().size());
|
|
|
+ Assert.assertEquals(Resource.newInstance(2 * GB, 1),
|
|
|
+ response.getContainersToUpdate().get(0).getResource());
|
|
|
+
|
|
|
+ //Decrease resources
|
|
|
+ allocateResponse = am1.sendContainerUpdateRequest(Arrays.asList(
|
|
|
+ UpdateContainerRequest.newInstance(2, container.getId(),
|
|
|
+ ContainerUpdateType.DECREASE_RESOURCE,
|
|
|
+ Resources.createResource(1 * GB, 1), null)));
|
|
|
+ Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
|
|
|
+
|
|
|
+ // Check that the container resources are decreased
|
|
|
+ // in NM through NM heartbeat response
|
|
|
+ response = nm1.nodeHeartbeat(true);
|
|
|
+ Assert.assertEquals(1, response.getContainersToUpdate().size());
|
|
|
+ Assert.assertEquals(Resource.newInstance(1 * GB, 1),
|
|
|
+ response.getContainersToUpdate().get(0).getResource());
|
|
|
+
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
+ // DEMOTE the container
|
|
|
+ allocateResponse = am1.sendContainerUpdateRequest(Arrays.asList(
|
|
|
+ UpdateContainerRequest.newInstance(3, container.getId(),
|
|
|
+ ContainerUpdateType.DEMOTE_EXECUTION_TYPE, null,
|
|
|
+ ExecutionType.OPPORTUNISTIC)));
|
|
|
+
|
|
|
+ response = nm1.nodeHeartbeat(Arrays.asList(ContainerStatus
|
|
|
+ .newInstance(container.getId(), ExecutionType.GUARANTEED,
|
|
|
+ ContainerState.RUNNING, "", 0)), true);
|
|
|
+ Thread.sleep(200);
|
|
|
+ if (allocateResponse.getUpdatedContainers().size() == 0) {
|
|
|
+ // Get the update response on next allocate
|
|
|
+ allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
|
|
|
+ }
|
|
|
+ // Check the update response from YARNRM
|
|
|
+ Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
|
|
|
+ uc = allocateResponse.getUpdatedContainers().get(0);
|
|
|
+ Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
|
|
|
+ uc.getContainer().getExecutionType());
|
|
|
+ // Check that the container is updated in NM through NM heartbeat response
|
|
|
+ if (response.getContainersToUpdate().size() == 0) {
|
|
|
+ response = nm1.nodeHeartbeat(true);
|
|
|
+ }
|
|
|
+ Assert.assertEquals(1, response.getContainersToUpdate().size());
|
|
|
+ Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
|
|
|
+ response.getContainersToUpdate().get(0).getExecutionType());
|
|
|
+ }
|
|
|
+
|
|
|
private void verifyMetrics(QueueMetrics metrics, long availableMB,
|
|
|
int availableVirtualCores, long allocatedMB,
|
|
|
int allocatedVirtualCores, int allocatedContainers) {
|