|
@@ -116,6 +116,7 @@ import org.junit.After;
|
|
import org.junit.Assert;
|
|
import org.junit.Assert;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
+import org.mockito.Mock;
|
|
import org.mockito.Mockito;
|
|
import org.mockito.Mockito;
|
|
import org.xml.sax.SAXException;
|
|
import org.xml.sax.SAXException;
|
|
|
|
|
|
@@ -2619,71 +2620,58 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|
assertEquals(1, scheduler.getSchedulerApp(attId4).getLiveContainers().size());
|
|
assertEquals(1, scheduler.getSchedulerApp(attId4).getLiveContainers().size());
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Reserve at a lower priority and verify the lower priority request gets
|
|
|
|
+ * allocated
|
|
|
|
+ */
|
|
@Test (timeout = 5000)
|
|
@Test (timeout = 5000)
|
|
- public void testReservationWhileMultiplePriorities() throws IOException {
|
|
|
|
|
|
+ public void testReservationWithMultiplePriorities() throws IOException {
|
|
scheduler.init(conf);
|
|
scheduler.init(conf);
|
|
scheduler.start();
|
|
scheduler.start();
|
|
scheduler.reinitialize(conf, resourceManager.getRMContext());
|
|
scheduler.reinitialize(conf, resourceManager.getRMContext());
|
|
|
|
|
|
// Add a node
|
|
// Add a node
|
|
- RMNode node1 =
|
|
|
|
- MockNodes
|
|
|
|
- .newNodeInfo(1, Resources.createResource(1024, 4), 1, "127.0.0.1");
|
|
|
|
|
|
+ RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(2048, 2));
|
|
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
|
|
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
|
|
scheduler.handle(nodeEvent1);
|
|
scheduler.handle(nodeEvent1);
|
|
-
|
|
|
|
- ApplicationAttemptId attId = createSchedulingRequest(1024, 4, "queue1",
|
|
|
|
- "user1", 1, 2);
|
|
|
|
- scheduler.update();
|
|
|
|
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
|
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
|
- scheduler.handle(updateEvent);
|
|
|
|
-
|
|
|
|
- FSAppAttempt app = scheduler.getSchedulerApp(attId);
|
|
|
|
- assertEquals(1, app.getLiveContainers().size());
|
|
|
|
-
|
|
|
|
- ContainerId containerId = scheduler.getSchedulerApp(attId)
|
|
|
|
- .getLiveContainers().iterator().next().getContainerId();
|
|
|
|
|
|
|
|
- // Cause reservation to be created
|
|
|
|
- createSchedulingRequestExistingApplication(1024, 4, 2, attId);
|
|
|
|
|
|
+ // Create first app and take up half resources so the second app that asks
|
|
|
|
+ // for the entire node won't have enough.
|
|
|
|
+ FSAppAttempt app1 = scheduler.getSchedulerApp(
|
|
|
|
+ createSchedulingRequest(1024, 1, "queue", "user", 1));
|
|
scheduler.update();
|
|
scheduler.update();
|
|
scheduler.handle(updateEvent);
|
|
scheduler.handle(updateEvent);
|
|
|
|
+ assertEquals("Basic allocation failed", 1, app1.getLiveContainers().size());
|
|
|
|
|
|
- assertEquals(1, app.getLiveContainers().size());
|
|
|
|
- assertEquals(0, scheduler.getRootQueueMetrics().getAvailableMB());
|
|
|
|
- assertEquals(0, scheduler.getRootQueueMetrics().getAvailableVirtualCores());
|
|
|
|
-
|
|
|
|
- // Create request at higher priority
|
|
|
|
- createSchedulingRequestExistingApplication(1024, 4, 1, attId);
|
|
|
|
|
|
+ // Create another app and reserve at a lower priority first
|
|
|
|
+ ApplicationAttemptId attId =
|
|
|
|
+ createSchedulingRequest(2048, 2, "queue1", "user1", 1, 2);
|
|
|
|
+ FSAppAttempt app2 = scheduler.getSchedulerApp(attId);
|
|
scheduler.update();
|
|
scheduler.update();
|
|
scheduler.handle(updateEvent);
|
|
scheduler.handle(updateEvent);
|
|
-
|
|
|
|
- assertEquals(1, app.getLiveContainers().size());
|
|
|
|
- // Reserved container should still be at lower priority
|
|
|
|
- for (RMContainer container : app.getReservedContainers()) {
|
|
|
|
- assertEquals(2,
|
|
|
|
- container.getReservedSchedulerKey().getPriority().getPriority());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Complete container
|
|
|
|
- scheduler.allocate(attId, new ArrayList<ResourceRequest>(),
|
|
|
|
|
|
+ assertEquals("Reservation at lower priority failed",
|
|
|
|
+ 1, app2.getReservedContainers().size());
|
|
|
|
+
|
|
|
|
+ // Request container on the second app at a higher priority
|
|
|
|
+ createSchedulingRequestExistingApplication(2048, 2, 1, attId);
|
|
|
|
+
|
|
|
|
+ // Complete the first container so we can trigger allocation for app2
|
|
|
|
+ ContainerId containerId =
|
|
|
|
+ app1.getLiveContainers().iterator().next().getContainerId();
|
|
|
|
+ scheduler.allocate(app1.getApplicationAttemptId(),
|
|
|
|
+ new ArrayList<ResourceRequest>(),
|
|
Arrays.asList(containerId), null, null, null, null);
|
|
Arrays.asList(containerId), null, null, null, null);
|
|
- assertEquals(1024, scheduler.getRootQueueMetrics().getAvailableMB());
|
|
|
|
- assertEquals(4, scheduler.getRootQueueMetrics().getAvailableVirtualCores());
|
|
|
|
-
|
|
|
|
- // Schedule at opening
|
|
|
|
- scheduler.update();
|
|
|
|
|
|
+
|
|
|
|
+ // Trigger allocation for app2
|
|
scheduler.handle(updateEvent);
|
|
scheduler.handle(updateEvent);
|
|
-
|
|
|
|
|
|
+
|
|
// Reserved container (at lower priority) should be run
|
|
// Reserved container (at lower priority) should be run
|
|
- Collection<RMContainer> liveContainers = app.getLiveContainers();
|
|
|
|
- assertEquals(1, liveContainers.size());
|
|
|
|
- for (RMContainer liveContainer : liveContainers) {
|
|
|
|
- Assert.assertEquals(2, liveContainer.getContainer().getPriority()
|
|
|
|
- .getPriority());
|
|
|
|
- }
|
|
|
|
- assertEquals(0, scheduler.getRootQueueMetrics().getAvailableMB());
|
|
|
|
- assertEquals(0, scheduler.getRootQueueMetrics().getAvailableVirtualCores());
|
|
|
|
|
|
+ Collection<RMContainer> liveContainers = app2.getLiveContainers();
|
|
|
|
+ assertEquals("Allocation post completion failed", 1, liveContainers.size());
|
|
|
|
+ assertEquals("High prio container allocated against low prio reservation",
|
|
|
|
+ 2, liveContainers.iterator().next().getContainer().
|
|
|
|
+ getPriority().getPriority());
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
@@ -3213,8 +3201,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * If we update our ask to strictly request a node, it doesn't make sense to keep
|
|
|
|
- * a reservation on another.
|
|
|
|
|
|
+ * Strict locality requests shouldn't reserve resources on another node.
|
|
*/
|
|
*/
|
|
@Test
|
|
@Test
|
|
public void testReservationsStrictLocality() throws IOException {
|
|
public void testReservationsStrictLocality() throws IOException {
|
|
@@ -3222,40 +3209,39 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|
scheduler.start();
|
|
scheduler.start();
|
|
scheduler.reinitialize(conf, resourceManager.getRMContext());
|
|
scheduler.reinitialize(conf, resourceManager.getRMContext());
|
|
|
|
|
|
- RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
|
|
|
|
- RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(1024), 2, "127.0.0.2");
|
|
|
|
|
|
+ // Add two nodes
|
|
|
|
+ RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024, 1));
|
|
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
|
|
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
|
|
scheduler.handle(nodeEvent1);
|
|
scheduler.handle(nodeEvent1);
|
|
|
|
+ RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(1024, 1));
|
|
|
|
+ NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
|
|
|
|
+ scheduler.handle(nodeEvent2);
|
|
|
|
|
|
- ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1",
|
|
|
|
- "user1", 0);
|
|
|
|
|
|
+ // Submit application without container requests
|
|
|
|
+ ApplicationAttemptId attId =
|
|
|
|
+ createSchedulingRequest(1024, "queue1", "user1", 0);
|
|
FSAppAttempt app = scheduler.getSchedulerApp(attId);
|
|
FSAppAttempt app = scheduler.getSchedulerApp(attId);
|
|
-
|
|
|
|
- ResourceRequest nodeRequest = createResourceRequest(1024, node2.getHostName(), 1, 2, true);
|
|
|
|
- ResourceRequest rackRequest = createResourceRequest(1024, "rack1", 1, 2, true);
|
|
|
|
- ResourceRequest anyRequest = createResourceRequest(1024, ResourceRequest.ANY,
|
|
|
|
- 1, 2, false);
|
|
|
|
|
|
+
|
|
|
|
+ // Request a container on node2
|
|
|
|
+ ResourceRequest nodeRequest =
|
|
|
|
+ createResourceRequest(1024, node2.getHostName(), 1, 1, true);
|
|
|
|
+ ResourceRequest rackRequest =
|
|
|
|
+ createResourceRequest(1024, "rack1", 1, 1, false);
|
|
|
|
+ ResourceRequest anyRequest =
|
|
|
|
+ createResourceRequest(1024, ResourceRequest.ANY, 1, 1, false);
|
|
createSchedulingRequestExistingApplication(nodeRequest, attId);
|
|
createSchedulingRequestExistingApplication(nodeRequest, attId);
|
|
createSchedulingRequestExistingApplication(rackRequest, attId);
|
|
createSchedulingRequestExistingApplication(rackRequest, attId);
|
|
createSchedulingRequestExistingApplication(anyRequest, attId);
|
|
createSchedulingRequestExistingApplication(anyRequest, attId);
|
|
-
|
|
|
|
scheduler.update();
|
|
scheduler.update();
|
|
|
|
|
|
|
|
+ // Heartbeat from node1. App shouldn't get an allocation or reservation
|
|
NodeUpdateSchedulerEvent nodeUpdateEvent = new NodeUpdateSchedulerEvent(node1);
|
|
NodeUpdateSchedulerEvent nodeUpdateEvent = new NodeUpdateSchedulerEvent(node1);
|
|
scheduler.handle(nodeUpdateEvent);
|
|
scheduler.handle(nodeUpdateEvent);
|
|
- assertEquals(1, app.getLiveContainers().size());
|
|
|
|
- scheduler.handle(nodeUpdateEvent);
|
|
|
|
- assertEquals(1, app.getReservedContainers().size());
|
|
|
|
-
|
|
|
|
- // now, make our request node-specific (on a different node)
|
|
|
|
- rackRequest = createResourceRequest(1024, "rack1", 1, 1, false);
|
|
|
|
- anyRequest = createResourceRequest(1024, ResourceRequest.ANY,
|
|
|
|
- 1, 1, false);
|
|
|
|
- scheduler.allocate(attId, Arrays.asList(rackRequest, anyRequest),
|
|
|
|
- new ArrayList<ContainerId>(), null, null, null, null);
|
|
|
|
-
|
|
|
|
|
|
+ assertEquals("App assigned a container on the wrong node",
|
|
|
|
+ 0, app.getLiveContainers().size());
|
|
scheduler.handle(nodeUpdateEvent);
|
|
scheduler.handle(nodeUpdateEvent);
|
|
- assertEquals(0, app.getReservedContainers().size());
|
|
|
|
|
|
+ assertEquals("App reserved a container on the wrong node",
|
|
|
|
+ 0, app.getReservedContainers().size());
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|