|
@@ -71,6 +71,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistributedSche
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistributedSchedulingAllocateResponsePBImpl;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterDistributedSchedulingAMResponsePBImpl;
|
|
|
import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
|
|
|
+import org.apache.hadoop.yarn.server.metrics.OpportunisticSchedulerMetrics;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
|
@@ -752,6 +753,106 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
|
Assert.assertEquals(allocatedContainers, metrics.getAllocatedContainers());
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testOpportunisticSchedulerMetrics() throws Exception {
|
|
|
+ HashMap<NodeId, MockNM> nodes = new HashMap<>();
|
|
|
+ MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm1.getNodeId(), nm1);
|
|
|
+ MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm2.getNodeId(), nm2);
|
|
|
+ nm1.registerNode();
|
|
|
+ nm2.registerNode();
|
|
|
+ OpportunisticSchedulerMetrics metrics =
|
|
|
+ OpportunisticSchedulerMetrics.getMetrics();
|
|
|
+
|
|
|
+ int allocContainers = metrics.getAllocatedContainers();
|
|
|
+ long aggrAllocatedContainers = metrics.getAggregatedAllocatedContainers();
|
|
|
+ long aggrOffSwitchContainers = metrics.getAggregatedOffSwitchContainers();
|
|
|
+ long aggrReleasedContainers = metrics.getAggregatedReleasedContainers();
|
|
|
+
|
|
|
+ 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, nm2);
|
|
|
+ ResourceScheduler scheduler = rm.getResourceScheduler();
|
|
|
+ RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
|
|
|
+ RMNode rmNode2 = rm.getRMContext().getRMNodes().get(nm2.getNodeId());
|
|
|
+
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
+ nm2.nodeHeartbeat(true);
|
|
|
+
|
|
|
+ ((RMNodeImpl) rmNode1)
|
|
|
+ .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
|
|
+ ((RMNodeImpl) rmNode2)
|
|
|
+ .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 NodeAddedSchedulerEvent(rmNode2));
|
|
|
+ amservice.handle(new NodeUpdateSchedulerEvent(rmNode1));
|
|
|
+ amservice.handle(new NodeUpdateSchedulerEvent(rmNode2));
|
|
|
+
|
|
|
+ // All nodes 1 to 2 will be applicable for scheduling.
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
+ nm2.nodeHeartbeat(true);
|
|
|
+
|
|
|
+ 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());
|
|
|
+
|
|
|
+ Assert.assertEquals(allocContainers + 2, metrics.getAllocatedContainers());
|
|
|
+ Assert.assertEquals(aggrAllocatedContainers + 2,
|
|
|
+ metrics.getAggregatedAllocatedContainers());
|
|
|
+ Assert.assertEquals(aggrOffSwitchContainers + 2,
|
|
|
+ metrics.getAggregatedOffSwitchContainers());
|
|
|
+
|
|
|
+ Container container = allocatedContainers.get(0);
|
|
|
+ MockNM allocNode = nodes.get(container.getNodeId());
|
|
|
+
|
|
|
+ // Start Container in NM
|
|
|
+ allocNode.nodeHeartbeat(Arrays.asList(
|
|
|
+ ContainerStatus.newInstance(container.getId(),
|
|
|
+ ExecutionType.OPPORTUNISTIC, ContainerState.RUNNING, "", 0)),
|
|
|
+ true);
|
|
|
+ rm.drainEvents();
|
|
|
+
|
|
|
+ // 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());
|
|
|
+
|
|
|
+ // Container Completed in the NM
|
|
|
+ allocNode.nodeHeartbeat(Arrays.asList(
|
|
|
+ ContainerStatus.newInstance(container.getId(),
|
|
|
+ ExecutionType.OPPORTUNISTIC, ContainerState.COMPLETE, "", 0)),
|
|
|
+ true);
|
|
|
+ rm.drainEvents();
|
|
|
+
|
|
|
+ // Verify that container has been removed..
|
|
|
+ rmContainer = ((CapacityScheduler) scheduler)
|
|
|
+ .getApplicationAttempt(
|
|
|
+ container.getId().getApplicationAttemptId()).getRMContainer(
|
|
|
+ container.getId());
|
|
|
+ Assert.assertNull(rmContainer);
|
|
|
+
|
|
|
+ Assert.assertEquals(allocContainers + 1, metrics.getAllocatedContainers());
|
|
|
+ Assert.assertEquals(aggrReleasedContainers + 1,
|
|
|
+ metrics.getAggregatedReleasedContainers());
|
|
|
+ }
|
|
|
+
|
|
|
@Test(timeout = 60000)
|
|
|
public void testAMCrashDuringAllocate() throws Exception {
|
|
|
MockNM nm = new MockNM("h:1234", 4096, rm.getResourceTrackerService());
|