|
@@ -488,6 +488,8 @@ public class TestRMContainerAllocator {
|
|
|
Configuration conf = new Configuration();
|
|
|
conf.setBoolean(MRJobConfig.MR_AM_JOB_NODE_BLACKLISTING_ENABLE, true);
|
|
|
conf.setInt(MRJobConfig.MAX_TASK_FAILURES_PER_TRACKER, 1);
|
|
|
+ conf.setInt(
|
|
|
+ MRJobConfig.MR_AM_IGNORE_BLACKLISTING_BLACKLISTED_NODE_PERECENT, -1);
|
|
|
|
|
|
MyResourceManager rm = new MyResourceManager(conf);
|
|
|
rm.start();
|
|
@@ -580,6 +582,175 @@ public class TestRMContainerAllocator {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testIgnoreBlacklisting() throws Exception {
|
|
|
+ LOG.info("Running testIgnoreBlacklisting");
|
|
|
+
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setBoolean(MRJobConfig.MR_AM_JOB_NODE_BLACKLISTING_ENABLE, true);
|
|
|
+ conf.setInt(MRJobConfig.MAX_TASK_FAILURES_PER_TRACKER, 1);
|
|
|
+ conf.setInt(
|
|
|
+ MRJobConfig.MR_AM_IGNORE_BLACKLISTING_BLACKLISTED_NODE_PERECENT, 33);
|
|
|
+
|
|
|
+ MyResourceManager rm = new MyResourceManager(conf);
|
|
|
+ rm.start();
|
|
|
+ DrainDispatcher dispatcher =
|
|
|
+ (DrainDispatcher) rm.getRMContext().getDispatcher();
|
|
|
+
|
|
|
+ // Submit the application
|
|
|
+ RMApp app = rm.submitApp(1024);
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ MockNM[] nodeManagers = new MockNM[10];
|
|
|
+ int nmNum = 0;
|
|
|
+ List<TaskAttemptContainerAssignedEvent> assigned = null;
|
|
|
+ nodeManagers[nmNum] = registerNodeManager(nmNum++, rm, dispatcher);
|
|
|
+ nodeManagers[0].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, false));
|
|
|
+ MyContainerAllocator allocator =
|
|
|
+ new MyContainerAllocator(rm, conf, appAttemptId, mockJob);
|
|
|
+
|
|
|
+ // Known=1, blacklisted=0, ignore should be false - assign first container
|
|
|
+ assigned =
|
|
|
+ getContainerOnHost(jobId, 1, 1024, new String[] { "h1" },
|
|
|
+ nodeManagers[0], dispatcher, allocator);
|
|
|
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
|
|
|
+
|
|
|
+ LOG.info("Failing container _1 on H1 (Node should be blacklisted and"
|
|
|
+ + " ignore blacklisting enabled");
|
|
|
+ // Send events to blacklist nodes h1 and h2
|
|
|
+ ContainerFailedEvent f1 = createFailEvent(jobId, 1, "h1", false);
|
|
|
+ allocator.sendFailure(f1);
|
|
|
+
|
|
|
+ // Test single node.
|
|
|
+ // Known=1, blacklisted=1, ignore should be true - assign 1
|
|
|
+ assigned =
|
|
|
+ getContainerOnHost(jobId, 2, 1024, new String[] { "h1" },
|
|
|
+ nodeManagers[0], dispatcher, allocator);
|
|
|
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
|
|
|
+
|
|
|
+ nodeManagers[nmNum] = registerNodeManager(nmNum++, rm, dispatcher);
|
|
|
+ // Known=2, blacklisted=1, ignore should be true - assign 1 anyway.
|
|
|
+ assigned =
|
|
|
+ getContainerOnHost(jobId, 3, 1024, new String[] { "h2" },
|
|
|
+ nodeManagers[1], dispatcher, allocator);
|
|
|
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
|
|
|
+
|
|
|
+ nodeManagers[nmNum] = registerNodeManager(nmNum++, rm, dispatcher);
|
|
|
+ // Known=3, blacklisted=1, ignore should be true - assign 1 anyway.
|
|
|
+ assigned =
|
|
|
+ getContainerOnHost(jobId, 4, 1024, new String[] { "h3" },
|
|
|
+ nodeManagers[2], dispatcher, allocator);
|
|
|
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
|
|
|
+
|
|
|
+ // Known=3, blacklisted=1, ignore should be true - assign 1
|
|
|
+ assigned =
|
|
|
+ getContainerOnHost(jobId, 5, 1024, new String[] { "h1" },
|
|
|
+ nodeManagers[0], dispatcher, allocator);
|
|
|
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
|
|
|
+
|
|
|
+ nodeManagers[nmNum] = registerNodeManager(nmNum++, rm, dispatcher);
|
|
|
+ // Known=4, blacklisted=1, ignore should be false - assign 1 anyway
|
|
|
+ assigned =
|
|
|
+ getContainerOnHost(jobId, 6, 1024, new String[] { "h4" },
|
|
|
+ nodeManagers[3], dispatcher, allocator);
|
|
|
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
|
|
|
+
|
|
|
+ // Test blacklisting re-enabled.
|
|
|
+ // Known=4, blacklisted=1, ignore should be false - no assignment on h1
|
|
|
+ assigned =
|
|
|
+ getContainerOnHost(jobId, 7, 1024, new String[] { "h1" },
|
|
|
+ nodeManagers[0], dispatcher, allocator);
|
|
|
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
|
|
|
+ // RMContainerRequestor would have created a replacement request.
|
|
|
+
|
|
|
+ // Blacklist h2
|
|
|
+ ContainerFailedEvent f2 = createFailEvent(jobId, 3, "h2", false);
|
|
|
+ allocator.sendFailure(f2);
|
|
|
+
|
|
|
+ // Test ignore blacklisting re-enabled
|
|
|
+ // Known=4, blacklisted=2, ignore should be true. Should assign 2
|
|
|
+ // containers.
|
|
|
+ assigned =
|
|
|
+ getContainerOnHost(jobId, 8, 1024, new String[] { "h1" },
|
|
|
+ nodeManagers[0], dispatcher, allocator);
|
|
|
+ Assert.assertEquals("No of assignments must be 2", 2, assigned.size());
|
|
|
+
|
|
|
+ // Known=4, blacklisted=2, ignore should be true.
|
|
|
+ assigned =
|
|
|
+ getContainerOnHost(jobId, 9, 1024, new String[] { "h2" },
|
|
|
+ nodeManagers[1], dispatcher, allocator);
|
|
|
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
|
|
|
+
|
|
|
+ // Test blacklist while ignore blacklisting enabled
|
|
|
+ ContainerFailedEvent f3 = createFailEvent(jobId, 4, "h3", false);
|
|
|
+ allocator.sendFailure(f3);
|
|
|
+
|
|
|
+ nodeManagers[nmNum] = registerNodeManager(nmNum++, rm, dispatcher);
|
|
|
+ // Known=5, blacklisted=3, ignore should be true.
|
|
|
+ assigned =
|
|
|
+ getContainerOnHost(jobId, 10, 1024, new String[] { "h3" },
|
|
|
+ nodeManagers[2], dispatcher, allocator);
|
|
|
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
|
|
|
+
|
|
|
+ // Assign on 5 more nodes - to re-enable blacklisting
|
|
|
+ for (int i = 0; i < 5; i++) {
|
|
|
+ nodeManagers[nmNum] = registerNodeManager(nmNum++, rm, dispatcher);
|
|
|
+ assigned =
|
|
|
+ getContainerOnHost(jobId, 11 + i, 1024,
|
|
|
+ new String[] { String.valueOf(5 + i) }, nodeManagers[4 + i],
|
|
|
+ dispatcher, allocator);
|
|
|
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
|
|
|
+ }
|
|
|
+
|
|
|
+ // Test h3 (blacklisted while ignoring blacklisting) is blacklisted.
|
|
|
+ assigned =
|
|
|
+ getContainerOnHost(jobId, 20, 1024, new String[] { "h3" },
|
|
|
+ nodeManagers[2], dispatcher, allocator);
|
|
|
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
|
|
|
+ }
|
|
|
+
|
|
|
+ private MockNM registerNodeManager(int i, MyResourceManager rm,
|
|
|
+ DrainDispatcher dispatcher) throws Exception {
|
|
|
+ MockNM nm = rm.registerNode("h" + (i + 1) + ":1234", 10240);
|
|
|
+ dispatcher.await();
|
|
|
+ return nm;
|
|
|
+ }
|
|
|
+
|
|
|
+ private
|
|
|
+ List<TaskAttemptContainerAssignedEvent> getContainerOnHost(JobId jobId,
|
|
|
+ int taskAttemptId, int memory, String[] hosts, MockNM mockNM,
|
|
|
+ DrainDispatcher dispatcher, MyContainerAllocator allocator)
|
|
|
+ throws Exception {
|
|
|
+ ContainerRequestEvent reqEvent =
|
|
|
+ createReq(jobId, taskAttemptId, memory, hosts);
|
|
|
+ allocator.sendRequest(reqEvent);
|
|
|
+
|
|
|
+ // Send the request to the RM
|
|
|
+ List<TaskAttemptContainerAssignedEvent> assigned = allocator.schedule();
|
|
|
+ dispatcher.await();
|
|
|
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
|
|
|
+
|
|
|
+ // Heartbeat from the required nodeManager
|
|
|
+ mockNM.nodeHeartbeat(true);
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ assigned = allocator.schedule();
|
|
|
+ dispatcher.await();
|
|
|
+ return assigned;
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testBlackListedNodesWithSchedulingToThatNode() throws Exception {
|
|
|
LOG.info("Running testBlackListedNodesWithSchedulingToThatNode");
|
|
@@ -587,6 +758,8 @@ public class TestRMContainerAllocator {
|
|
|
Configuration conf = new Configuration();
|
|
|
conf.setBoolean(MRJobConfig.MR_AM_JOB_NODE_BLACKLISTING_ENABLE, true);
|
|
|
conf.setInt(MRJobConfig.MAX_TASK_FAILURES_PER_TRACKER, 1);
|
|
|
+ conf.setInt(
|
|
|
+ MRJobConfig.MR_AM_IGNORE_BLACKLISTING_BLACKLISTED_NODE_PERECENT, -1);
|
|
|
|
|
|
MyResourceManager rm = new MyResourceManager(conf);
|
|
|
rm.start();
|