|
@@ -233,6 +233,17 @@ public class TestCapacityScheduler {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private NodeManager registerNode(ResourceManager rm, String hostName,
|
|
|
+ int containerManagerPort, int httpPort, String rackName,
|
|
|
+ Resource capability) throws IOException, YarnException {
|
|
|
+ NodeManager nm = new NodeManager(hostName,
|
|
|
+ containerManagerPort, httpPort, rackName, capability, rm);
|
|
|
+ NodeAddedSchedulerEvent nodeAddEvent1 =
|
|
|
+ new NodeAddedSchedulerEvent(rm.getRMContext().getRMNodes()
|
|
|
+ .get(nm.getNodeId()));
|
|
|
+ rm.getResourceScheduler().handle(nodeAddEvent1);
|
|
|
+ return nm;
|
|
|
+ }
|
|
|
|
|
|
@Test (timeout = 30000)
|
|
|
public void testConfValidation() throws Exception {
|
|
@@ -267,12 +278,12 @@ public class TestCapacityScheduler {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
|
|
|
+ private NodeManager
|
|
|
registerNode(String hostName, int containerManagerPort, int httpPort,
|
|
|
String rackName, Resource capability)
|
|
|
throws IOException, YarnException {
|
|
|
- org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm =
|
|
|
- new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
|
|
|
+ NodeManager nm =
|
|
|
+ new NodeManager(
|
|
|
hostName, containerManagerPort, httpPort, rackName, capability,
|
|
|
resourceManager);
|
|
|
NodeAddedSchedulerEvent nodeAddEvent1 =
|
|
@@ -400,8 +411,216 @@ public class TestCapacityScheduler {
|
|
|
LOG.info("--- END: testCapacityScheduler ---");
|
|
|
}
|
|
|
|
|
|
- private void nodeUpdate(
|
|
|
- org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm) {
|
|
|
+ @Test
|
|
|
+ public void testNotAssignMultiple() throws Exception {
|
|
|
+ LOG.info("--- START: testNotAssignMultiple ---");
|
|
|
+ ResourceManager rm = new ResourceManager() {
|
|
|
+ @Override
|
|
|
+ protected RMNodeLabelsManager createNodeLabelManager() {
|
|
|
+ RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
|
|
|
+ mgr.init(getConfig());
|
|
|
+ return mgr;
|
|
|
+ }
|
|
|
+ };
|
|
|
+ CapacitySchedulerConfiguration csConf =
|
|
|
+ new CapacitySchedulerConfiguration();
|
|
|
+ csConf.setBoolean(
|
|
|
+ CapacitySchedulerConfiguration.ASSIGN_MULTIPLE_ENABLED, false);
|
|
|
+ setupQueueConfiguration(csConf);
|
|
|
+ YarnConfiguration conf = new YarnConfiguration(csConf);
|
|
|
+ conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
|
|
|
+ ResourceScheduler.class);
|
|
|
+ rm.init(conf);
|
|
|
+ rm.getRMContext().getContainerTokenSecretManager().rollMasterKey();
|
|
|
+ rm.getRMContext().getNMTokenSecretManager().rollMasterKey();
|
|
|
+ ((AsyncDispatcher) rm.getRMContext().getDispatcher()).start();
|
|
|
+ RMContext mC = mock(RMContext.class);
|
|
|
+ when(mC.getConfigurationProvider()).thenReturn(
|
|
|
+ new LocalConfigurationProvider());
|
|
|
+
|
|
|
+ // Register node1
|
|
|
+ String host0 = "host_0";
|
|
|
+ NodeManager nm0 =
|
|
|
+ registerNode(rm, host0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
|
|
+ Resources.createResource(10 * GB, 10));
|
|
|
+
|
|
|
+ // ResourceRequest priorities
|
|
|
+ Priority priority0 = Priority.newInstance(0);
|
|
|
+ Priority priority1 = Priority.newInstance(1);
|
|
|
+
|
|
|
+ // Submit an application
|
|
|
+ Application application0 = new Application("user_0", "a1", rm);
|
|
|
+ application0.submit();
|
|
|
+ application0.addNodeManager(host0, 1234, nm0);
|
|
|
+
|
|
|
+ Resource capability00 = Resources.createResource(1 * GB, 1);
|
|
|
+ application0.addResourceRequestSpec(priority0, capability00);
|
|
|
+
|
|
|
+ Resource capability01 = Resources.createResource(2 * GB, 1);
|
|
|
+ application0.addResourceRequestSpec(priority1, capability01);
|
|
|
+
|
|
|
+ Task task00 =
|
|
|
+ new Task(application0, priority0, new String[] {host0});
|
|
|
+ Task task01 =
|
|
|
+ new Task(application0, priority1, new String[] {host0});
|
|
|
+ application0.addTask(task00);
|
|
|
+ application0.addTask(task01);
|
|
|
+
|
|
|
+ // Submit another application
|
|
|
+ Application application1 = new Application("user_1", "b2", rm);
|
|
|
+ application1.submit();
|
|
|
+ application1.addNodeManager(host0, 1234, nm0);
|
|
|
+
|
|
|
+ Resource capability10 = Resources.createResource(3 * GB, 1);
|
|
|
+ application1.addResourceRequestSpec(priority0, capability10);
|
|
|
+
|
|
|
+ Resource capability11 = Resources.createResource(4 * GB, 1);
|
|
|
+ application1.addResourceRequestSpec(priority1, capability11);
|
|
|
+
|
|
|
+ Task task10 = new Task(application1, priority0, new String[] {host0});
|
|
|
+ Task task11 = new Task(application1, priority1, new String[] {host0});
|
|
|
+ application1.addTask(task10);
|
|
|
+ application1.addTask(task11);
|
|
|
+
|
|
|
+ // Send resource requests to the scheduler
|
|
|
+ application0.schedule();
|
|
|
+
|
|
|
+ application1.schedule();
|
|
|
+
|
|
|
+ // Send a heartbeat to kick the tires on the Scheduler
|
|
|
+ LOG.info("Kick!");
|
|
|
+
|
|
|
+ // task00, used=1G
|
|
|
+ nodeUpdate(rm, nm0);
|
|
|
+
|
|
|
+ // Get allocations from the scheduler
|
|
|
+ application0.schedule();
|
|
|
+ application1.schedule();
|
|
|
+ // 1 Task per heart beat should be scheduled
|
|
|
+ checkNodeResourceUsage(3 * GB, nm0); // task00 (1G)
|
|
|
+ checkApplicationResourceUsage(0 * GB, application0);
|
|
|
+ checkApplicationResourceUsage(3 * GB, application1);
|
|
|
+
|
|
|
+ // Another heartbeat
|
|
|
+ nodeUpdate(rm, nm0);
|
|
|
+ application0.schedule();
|
|
|
+ checkApplicationResourceUsage(1 * GB, application0);
|
|
|
+ application1.schedule();
|
|
|
+ checkApplicationResourceUsage(3 * GB, application1);
|
|
|
+ checkNodeResourceUsage(4 * GB, nm0);
|
|
|
+ LOG.info("--- START: testNotAssignMultiple ---");
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testAssignMultiple() throws Exception {
|
|
|
+ LOG.info("--- START: testAssignMultiple ---");
|
|
|
+ ResourceManager rm = new ResourceManager() {
|
|
|
+ @Override
|
|
|
+ protected RMNodeLabelsManager createNodeLabelManager() {
|
|
|
+ RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
|
|
|
+ mgr.init(getConfig());
|
|
|
+ return mgr;
|
|
|
+ }
|
|
|
+ };
|
|
|
+ CapacitySchedulerConfiguration csConf =
|
|
|
+ new CapacitySchedulerConfiguration();
|
|
|
+ csConf.setBoolean(
|
|
|
+ CapacitySchedulerConfiguration.ASSIGN_MULTIPLE_ENABLED, true);
|
|
|
+ // Each heartbeat will assign 2 containers at most
|
|
|
+ csConf.setInt(CapacitySchedulerConfiguration.MAX_ASSIGN_PER_HEARTBEAT, 2);
|
|
|
+ setupQueueConfiguration(csConf);
|
|
|
+ YarnConfiguration conf = new YarnConfiguration(csConf);
|
|
|
+ conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
|
|
|
+ ResourceScheduler.class);
|
|
|
+ rm.init(conf);
|
|
|
+ rm.getRMContext().getContainerTokenSecretManager().rollMasterKey();
|
|
|
+ rm.getRMContext().getNMTokenSecretManager().rollMasterKey();
|
|
|
+ ((AsyncDispatcher) rm.getRMContext().getDispatcher()).start();
|
|
|
+ RMContext mC = mock(RMContext.class);
|
|
|
+ when(mC.getConfigurationProvider()).thenReturn(
|
|
|
+ new LocalConfigurationProvider());
|
|
|
+
|
|
|
+ // Register node1
|
|
|
+ String host0 = "host_0";
|
|
|
+ NodeManager nm0 =
|
|
|
+ registerNode(rm, host0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
|
|
+ Resources.createResource(10 * GB, 10));
|
|
|
+
|
|
|
+ // ResourceRequest priorities
|
|
|
+ Priority priority0 = Priority.newInstance(0);
|
|
|
+ Priority priority1 = Priority.newInstance(1);
|
|
|
+
|
|
|
+ // Submit an application
|
|
|
+ Application application0 = new Application("user_0", "a1", rm);
|
|
|
+ application0.submit();
|
|
|
+ application0.addNodeManager(host0, 1234, nm0);
|
|
|
+
|
|
|
+ Resource capability00 = Resources.createResource(1 * GB, 1);
|
|
|
+ application0.addResourceRequestSpec(priority0, capability00);
|
|
|
+
|
|
|
+ Resource capability01 = Resources.createResource(2 * GB, 1);
|
|
|
+ application0.addResourceRequestSpec(priority1, capability01);
|
|
|
+
|
|
|
+ Task task00 = new Task(application0, priority0, new String[] {host0});
|
|
|
+ Task task01 = new Task(application0, priority1, new String[] {host0});
|
|
|
+ application0.addTask(task00);
|
|
|
+ application0.addTask(task01);
|
|
|
+
|
|
|
+ // Submit another application
|
|
|
+ Application application1 = new Application("user_1", "b2", rm);
|
|
|
+ application1.submit();
|
|
|
+ application1.addNodeManager(host0, 1234, nm0);
|
|
|
+
|
|
|
+ Resource capability10 = Resources.createResource(3 * GB, 1);
|
|
|
+ application1.addResourceRequestSpec(priority0, capability10);
|
|
|
+
|
|
|
+ Resource capability11 = Resources.createResource(4 * GB, 1);
|
|
|
+ application1.addResourceRequestSpec(priority1, capability11);
|
|
|
+
|
|
|
+ Task task10 =
|
|
|
+ new Task(application1, priority0, new String[] {host0});
|
|
|
+ Task task11 =
|
|
|
+ new Task(application1, priority1, new String[] {host0});
|
|
|
+ application1.addTask(task10);
|
|
|
+ application1.addTask(task11);
|
|
|
+
|
|
|
+ // Send resource requests to the scheduler
|
|
|
+ application0.schedule();
|
|
|
+
|
|
|
+ application1.schedule();
|
|
|
+
|
|
|
+ // Send a heartbeat to kick the tires on the Scheduler
|
|
|
+ LOG.info("Kick!");
|
|
|
+
|
|
|
+ // task_0_0, used=1G
|
|
|
+ nodeUpdate(rm, nm0);
|
|
|
+
|
|
|
+ // Get allocations from the scheduler
|
|
|
+ application0.schedule();
|
|
|
+ application1.schedule();
|
|
|
+ // 1 Task per heart beat should be scheduled
|
|
|
+ checkNodeResourceUsage(4 * GB, nm0); // task00 (1G)
|
|
|
+ checkApplicationResourceUsage(1 * GB, application0);
|
|
|
+ checkApplicationResourceUsage(3 * GB, application1);
|
|
|
+
|
|
|
+ // Another heartbeat
|
|
|
+ nodeUpdate(rm, nm0);
|
|
|
+ application0.schedule();
|
|
|
+ checkApplicationResourceUsage(3 * GB, application0);
|
|
|
+ application1.schedule();
|
|
|
+ checkApplicationResourceUsage(7 * GB, application1);
|
|
|
+ checkNodeResourceUsage(10 * GB, nm0);
|
|
|
+ LOG.info("--- START: testAssignMultiple ---");
|
|
|
+ }
|
|
|
+
|
|
|
+ private void nodeUpdate(ResourceManager rm, NodeManager nm) {
|
|
|
+ RMNode node = rm.getRMContext().getRMNodes().get(nm.getNodeId());
|
|
|
+ // Send a heartbeat to kick the tires on the Scheduler
|
|
|
+ NodeUpdateSchedulerEvent nodeUpdate = new NodeUpdateSchedulerEvent(node);
|
|
|
+ rm.getResourceScheduler().handle(nodeUpdate);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void nodeUpdate(NodeManager nm) {
|
|
|
RMNode node = resourceManager.getRMContext().getRMNodes().get(nm.getNodeId());
|
|
|
// Send a heartbeat to kick the tires on the Scheduler
|
|
|
NodeUpdateSchedulerEvent nodeUpdate = new NodeUpdateSchedulerEvent(node);
|
|
@@ -699,8 +918,7 @@ public class TestCapacityScheduler {
|
|
|
Assert.assertEquals(expected, application.getUsedResources().getMemorySize());
|
|
|
}
|
|
|
|
|
|
- private void checkNodeResourceUsage(int expected,
|
|
|
- org.apache.hadoop.yarn.server.resourcemanager.NodeManager node) {
|
|
|
+ private void checkNodeResourceUsage(int expected, NodeManager node) {
|
|
|
Assert.assertEquals(expected, node.getUsed().getMemorySize());
|
|
|
node.checkResourceUsage();
|
|
|
}
|