|
@@ -580,6 +580,135 @@ public class TestRMContainerAllocator {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testBlackListedNodesWithSchedulingToThatNode() throws Exception {
|
|
|
+ LOG.info("Running testBlackListedNodesWithSchedulingToThatNode");
|
|
|
+
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setBoolean(MRJobConfig.MR_AM_JOB_NODE_BLACKLISTING_ENABLE, true);
|
|
|
+ conf.setInt(MRJobConfig.MAX_TASK_FAILURES_PER_TRACKER, 1);
|
|
|
+
|
|
|
+ MyResourceManager rm = new MyResourceManager(conf);
|
|
|
+ rm.start();
|
|
|
+ DrainDispatcher dispatcher = (DrainDispatcher) rm.getRMContext()
|
|
|
+ .getDispatcher();
|
|
|
+
|
|
|
+ // Submit the application
|
|
|
+ RMApp app = rm.submitApp(1024);
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
|
|
|
+ amNodeManager.nodeHeartbeat(true);
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt()
|
|
|
+ .getAppAttemptId();
|
|
|
+ rm.sendAMLaunched(appAttemptId);
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ JobId jobId = MRBuilderUtils.newJobId(appAttemptId.getApplicationId(), 0);
|
|
|
+ Job mockJob = mock(Job.class);
|
|
|
+ when(mockJob.getReport()).thenReturn(
|
|
|
+ MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
|
|
|
+ 0, 0, 0, 0, 0, 0, "jobfile", null));
|
|
|
+ MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
|
|
|
+ appAttemptId, mockJob);
|
|
|
+
|
|
|
+ // add resources to scheduler
|
|
|
+ MockNM nodeManager1 = rm.registerNode("h1:1234", 10240);
|
|
|
+ MockNM nodeManager3 = rm.registerNode("h3:1234", 10240);
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ LOG.info("Requesting 1 Containers _1 on H1");
|
|
|
+ // create the container request
|
|
|
+ ContainerRequestEvent event1 = createReq(jobId, 1, 1024,
|
|
|
+ new String[] { "h1" });
|
|
|
+ allocator.sendRequest(event1);
|
|
|
+
|
|
|
+ LOG.info("RM Heartbeat (to send the container requests)");
|
|
|
+ // this tells the scheduler about the requests
|
|
|
+ // as nodes are not added, no allocations
|
|
|
+ List<TaskAttemptContainerAssignedEvent> assigned = allocator.schedule();
|
|
|
+ dispatcher.await();
|
|
|
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
|
|
|
+
|
|
|
+ LOG.info("h1 Heartbeat (To actually schedule the containers)");
|
|
|
+ // update resources in scheduler
|
|
|
+ nodeManager1.nodeHeartbeat(true); // Node heartbeat
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ LOG.info("RM Heartbeat (To process the scheduled containers)");
|
|
|
+ assigned = allocator.schedule();
|
|
|
+ dispatcher.await();
|
|
|
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
|
|
|
+
|
|
|
+ LOG.info("Failing container _1 on H1 (should blacklist the node)");
|
|
|
+ // Send events to blacklist nodes h1 and h2
|
|
|
+ ContainerFailedEvent f1 = createFailEvent(jobId, 1, "h1", false);
|
|
|
+ allocator.sendFailure(f1);
|
|
|
+
|
|
|
+ //At this stage, a request should be created for a fast fail map
|
|
|
+ //Create a FAST_FAIL request for a previously failed map.
|
|
|
+ ContainerRequestEvent event1f = createReq(jobId, 1, 1024,
|
|
|
+ new String[] { "h1" }, true, false);
|
|
|
+ allocator.sendRequest(event1f);
|
|
|
+
|
|
|
+ //Update the Scheduler with the new requests.
|
|
|
+ assigned = allocator.schedule();
|
|
|
+ dispatcher.await();
|
|
|
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
|
|
|
+
|
|
|
+ // send another request with different resource and priority
|
|
|
+ ContainerRequestEvent event3 = createReq(jobId, 3, 1024,
|
|
|
+ new String[] { "h1", "h3" });
|
|
|
+ allocator.sendRequest(event3);
|
|
|
+
|
|
|
+ //Allocator is aware of prio:5 container, and prio:20 (h1+h3) container.
|
|
|
+ //RM is only aware of the prio:5 container
|
|
|
+
|
|
|
+ LOG.info("h1 Heartbeat (To actually schedule the containers)");
|
|
|
+ // update resources in scheduler
|
|
|
+ nodeManager1.nodeHeartbeat(true); // Node heartbeat
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ LOG.info("RM Heartbeat (To process the scheduled containers)");
|
|
|
+ assigned = allocator.schedule();
|
|
|
+ dispatcher.await();
|
|
|
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
|
|
|
+
|
|
|
+ //RMContainerAllocator gets assigned a p:5 on a blacklisted node.
|
|
|
+
|
|
|
+ //Send a release for the p:5 container + another request.
|
|
|
+ LOG.info("RM Heartbeat (To process the re-scheduled containers)");
|
|
|
+ assigned = allocator.schedule();
|
|
|
+ dispatcher.await();
|
|
|
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
|
|
|
+
|
|
|
+ //Hearbeat from H3 to schedule on this host.
|
|
|
+ LOG.info("h3 Heartbeat (To re-schedule the containers)");
|
|
|
+ nodeManager3.nodeHeartbeat(true); // Node heartbeat
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ LOG.info("RM Heartbeat (To process the re-scheduled containers for H3)");
|
|
|
+ assigned = allocator.schedule();
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ // For debugging
|
|
|
+ for (TaskAttemptContainerAssignedEvent assig : assigned) {
|
|
|
+ LOG.info(assig.getTaskAttemptID() +
|
|
|
+ " assgined to " + assig.getContainer().getId() +
|
|
|
+ " with priority " + assig.getContainer().getPriority());
|
|
|
+ }
|
|
|
+
|
|
|
+ Assert.assertEquals("No of assignments must be 2", 2, assigned.size());
|
|
|
+
|
|
|
+ // validate that all containers are assigned to h3
|
|
|
+ for (TaskAttemptContainerAssignedEvent assig : assigned) {
|
|
|
+ Assert.assertEquals("Assigned container " + assig.getContainer().getId()
|
|
|
+ + " host not correct", "h3", assig.getContainer().getNodeId().getHost());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private static class MyFifoScheduler extends FifoScheduler {
|
|
|
|
|
|
public MyFifoScheduler(RMContext rmContext) {
|