|
@@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaS
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
|
|
@@ -111,6 +112,8 @@ public class TestCapacityScheduler {
|
|
|
conf.setClass(YarnConfiguration.RM_SCHEDULER,
|
|
|
CapacityScheduler.class, ResourceScheduler.class);
|
|
|
resourceManager.init(conf);
|
|
|
+ resourceManager.getRMContainerTokenSecretManager().rollMasterKey();
|
|
|
+ resourceManager.getRMNMTokenSecretManager().rollMasterKey();
|
|
|
((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start();
|
|
|
}
|
|
|
|
|
@@ -156,13 +159,18 @@ public class TestCapacityScheduler {
|
|
|
registerNode(String hostName, int containerManagerPort, int httpPort,
|
|
|
String rackName, Resource capability)
|
|
|
throws IOException, YarnException {
|
|
|
- return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
|
|
|
- hostName, containerManagerPort, httpPort, rackName, capability,
|
|
|
- resourceManager.getResourceTrackerService(), resourceManager
|
|
|
- .getRMContext());
|
|
|
- }
|
|
|
+ org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm =
|
|
|
+ new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
|
|
|
+ hostName, containerManagerPort, httpPort, rackName, capability,
|
|
|
+ resourceManager);
|
|
|
+ NodeAddedSchedulerEvent nodeAddEvent1 =
|
|
|
+ new NodeAddedSchedulerEvent(resourceManager.getRMContext()
|
|
|
+ .getRMNodes().get(nm.getNodeId()));
|
|
|
+ resourceManager.getResourceScheduler().handle(nodeAddEvent1);
|
|
|
+ return nm;
|
|
|
+ }
|
|
|
|
|
|
-// @Test
|
|
|
+ @Test
|
|
|
public void testCapacityScheduler() throws Exception {
|
|
|
|
|
|
LOG.info("--- START: testCapacityScheduler ---");
|
|
@@ -172,14 +180,12 @@ public class TestCapacityScheduler {
|
|
|
org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_0 =
|
|
|
registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
|
|
Resources.createResource(4 * GB, 1));
|
|
|
- nm_0.heartbeat();
|
|
|
|
|
|
// Register node2
|
|
|
String host_1 = "host_1";
|
|
|
org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_1 =
|
|
|
registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
|
|
Resources.createResource(2 * GB, 1));
|
|
|
- nm_1.heartbeat();
|
|
|
|
|
|
// ResourceRequest priorities
|
|
|
Priority priority_0 =
|
|
@@ -227,9 +233,13 @@ public class TestCapacityScheduler {
|
|
|
|
|
|
// Send a heartbeat to kick the tires on the Scheduler
|
|
|
LOG.info("Kick!");
|
|
|
- nm_0.heartbeat(); // task_0_0 and task_1_0 allocated, used=4G
|
|
|
- nm_1.heartbeat(); // nothing allocated
|
|
|
-
|
|
|
+
|
|
|
+ // task_0_0 and task_1_0 allocated, used=4G
|
|
|
+ nodeUpdate(nm_0);
|
|
|
+
|
|
|
+ // nothing allocated
|
|
|
+ nodeUpdate(nm_1);
|
|
|
+
|
|
|
// Get allocations from the scheduler
|
|
|
application_0.schedule(); // task_0_0
|
|
|
checkApplicationResourceUsage(1 * GB, application_0);
|
|
@@ -237,9 +247,6 @@ public class TestCapacityScheduler {
|
|
|
application_1.schedule(); // task_1_0
|
|
|
checkApplicationResourceUsage(3 * GB, application_1);
|
|
|
|
|
|
- nm_0.heartbeat();
|
|
|
- nm_1.heartbeat();
|
|
|
-
|
|
|
checkNodeResourceUsage(4*GB, nm_0); // task_0_0 (1G) and task_1_0 (3G)
|
|
|
checkNodeResourceUsage(0*GB, nm_1); // no tasks, 2G available
|
|
|
|
|
@@ -259,10 +266,12 @@ public class TestCapacityScheduler {
|
|
|
|
|
|
// Send a heartbeat to kick the tires on the Scheduler
|
|
|
LOG.info("Sending hb from " + nm_0.getHostName());
|
|
|
- nm_0.heartbeat(); // nothing new, used=4G
|
|
|
+ // nothing new, used=4G
|
|
|
+ nodeUpdate(nm_0);
|
|
|
|
|
|
LOG.info("Sending hb from " + nm_1.getHostName());
|
|
|
- nm_1.heartbeat(); // task_0_3, used=2G
|
|
|
+ // task_0_1 is prefer as locality, used=2G
|
|
|
+ nodeUpdate(nm_1);
|
|
|
|
|
|
// Get allocations from the scheduler
|
|
|
LOG.info("Trying to allocate...");
|
|
@@ -272,13 +281,22 @@ public class TestCapacityScheduler {
|
|
|
application_1.schedule();
|
|
|
checkApplicationResourceUsage(5 * GB, application_1);
|
|
|
|
|
|
- nm_0.heartbeat();
|
|
|
- nm_1.heartbeat();
|
|
|
+ nodeUpdate(nm_0);
|
|
|
+ nodeUpdate(nm_1);
|
|
|
+
|
|
|
checkNodeResourceUsage(4*GB, nm_0);
|
|
|
checkNodeResourceUsage(2*GB, nm_1);
|
|
|
|
|
|
LOG.info("--- END: testCapacityScheduler ---");
|
|
|
}
|
|
|
+
|
|
|
+ private void nodeUpdate(
|
|
|
+ org.apache.hadoop.yarn.server.resourcemanager.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);
|
|
|
+ resourceManager.getResourceScheduler().handle(nodeUpdate);
|
|
|
+ }
|
|
|
|
|
|
private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) {
|
|
|
|