|
@@ -44,6 +44,8 @@ import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
|
|
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
|
|
|
|
+import org.apache.hadoop.yarn.event.Dispatcher;
|
|
|
|
+import org.apache.hadoop.yarn.event.DrainDispatcher;
|
|
import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocolPB;
|
|
import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocolPB;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
|
@@ -108,6 +110,7 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
private static final int GB = 1024;
|
|
private static final int GB = 1024;
|
|
|
|
|
|
private MockRM rm;
|
|
private MockRM rm;
|
|
|
|
+ private DrainDispatcher dispatcher;
|
|
|
|
|
|
@Before
|
|
@Before
|
|
public void createAndStartRM() {
|
|
public void createAndStartRM() {
|
|
@@ -120,8 +123,7 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
|
|
YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
|
|
conf.setInt(
|
|
conf.setInt(
|
|
YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS, 100);
|
|
YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS, 100);
|
|
- rm = new MockRM(conf);
|
|
|
|
- rm.start();
|
|
|
|
|
|
+ startRM(conf);
|
|
}
|
|
}
|
|
|
|
|
|
public void createAndStartRMWithAutoUpdateContainer() {
|
|
public void createAndStartRMWithAutoUpdateContainer() {
|
|
@@ -135,7 +137,17 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
|
|
YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
|
|
conf.setInt(
|
|
conf.setInt(
|
|
YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS, 100);
|
|
YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS, 100);
|
|
- rm = new MockRM(conf);
|
|
|
|
|
|
+ startRM(conf);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void startRM(final YarnConfiguration conf) {
|
|
|
|
+ dispatcher = new DrainDispatcher();
|
|
|
|
+ rm = new MockRM(conf) {
|
|
|
|
+ @Override
|
|
|
|
+ protected Dispatcher createDispatcher() {
|
|
|
|
+ return dispatcher;
|
|
|
|
+ }
|
|
|
|
+ };
|
|
rm.start();
|
|
rm.start();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -180,17 +192,6 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
nm3.nodeHeartbeat(true);
|
|
nm3.nodeHeartbeat(true);
|
|
nm4.nodeHeartbeat(true);
|
|
nm4.nodeHeartbeat(true);
|
|
|
|
|
|
- ((RMNodeImpl) rmNode1)
|
|
|
|
- .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
|
|
|
- ((RMNodeImpl) rmNode2)
|
|
|
|
- .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
|
|
|
- ((RMNodeImpl) rmNode3)
|
|
|
|
- .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
|
|
|
- ((RMNodeImpl) rmNode4)
|
|
|
|
- .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
|
|
|
-
|
|
|
|
- OpportunisticContainerContext ctxt = ((CapacityScheduler) scheduler)
|
|
|
|
- .getApplicationAttempt(attemptId).getOpportunisticContainerContext();
|
|
|
|
// Send add and update node events to AM Service.
|
|
// Send add and update node events to AM Service.
|
|
amservice.handle(new NodeAddedSchedulerEvent(rmNode1));
|
|
amservice.handle(new NodeAddedSchedulerEvent(rmNode1));
|
|
amservice.handle(new NodeAddedSchedulerEvent(rmNode2));
|
|
amservice.handle(new NodeAddedSchedulerEvent(rmNode2));
|
|
@@ -246,6 +247,9 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
|
|
allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
|
|
Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
|
|
Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
|
|
|
|
|
|
|
|
+ // Wait for scheduler to process all events
|
|
|
|
+ dispatcher.waitForEventThreadToWait();
|
|
|
|
+ Thread.sleep(1000);
|
|
// Verify Metrics After OPP allocation (Nothing should change again)
|
|
// Verify Metrics After OPP allocation (Nothing should change again)
|
|
verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
|
|
verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
|
|
|
|
|
|
@@ -319,6 +323,8 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
Assert.assertEquals(uc.getId(), container.getId());
|
|
Assert.assertEquals(uc.getId(), container.getId());
|
|
Assert.assertEquals(uc.getVersion(), container.getVersion() + 2);
|
|
Assert.assertEquals(uc.getVersion(), container.getVersion() + 2);
|
|
|
|
|
|
|
|
+ // Wait for scheduler to finish processing events
|
|
|
|
+ dispatcher.waitForEventThreadToWait();
|
|
// Verify Metrics After OPP allocation :
|
|
// Verify Metrics After OPP allocation :
|
|
// Everything should have reverted to what it was
|
|
// Everything should have reverted to what it was
|
|
verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
|
|
verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
|
|
@@ -663,6 +669,7 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
Assert.assertEquals(container.getId(), uc.getContainer().getId());
|
|
Assert.assertEquals(container.getId(), uc.getContainer().getId());
|
|
Assert.assertEquals(Resource.newInstance(2 * GB, 1),
|
|
Assert.assertEquals(Resource.newInstance(2 * GB, 1),
|
|
uc.getContainer().getResource());
|
|
uc.getContainer().getResource());
|
|
|
|
+ Thread.sleep(1000);
|
|
|
|
|
|
// Check that the container resources are increased in
|
|
// Check that the container resources are increased in
|
|
// NM through NM heartbeat response
|
|
// NM through NM heartbeat response
|
|
@@ -679,6 +686,7 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
ContainerUpdateType.DECREASE_RESOURCE,
|
|
ContainerUpdateType.DECREASE_RESOURCE,
|
|
Resources.createResource(1 * GB, 1), null)));
|
|
Resources.createResource(1 * GB, 1), null)));
|
|
Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
|
|
Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
|
|
|
|
+ Thread.sleep(1000);
|
|
|
|
|
|
// Check that the container resources are decreased
|
|
// Check that the container resources are decreased
|
|
// in NM through NM heartbeat response
|
|
// in NM through NM heartbeat response
|