|
@@ -30,7 +30,6 @@ import java.io.PrintWriter;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
|
-import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
@@ -267,7 +266,7 @@ public class TestFairScheduler {
|
|
|
Assert.assertEquals(3, queueManager.getLeafQueues().size());
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test (timeout = 5000)
|
|
|
public void testSimpleContainerAllocation() {
|
|
|
// Add a node
|
|
|
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
|
|
@@ -283,8 +282,7 @@ public class TestFairScheduler {
|
|
|
|
|
|
scheduler.update();
|
|
|
|
|
|
- NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1,
|
|
|
- new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
|
|
scheduler.handle(updateEvent);
|
|
|
|
|
|
// Asked for less than min_allocation.
|
|
@@ -292,15 +290,14 @@ public class TestFairScheduler {
|
|
|
scheduler.getQueueManager().getQueue("queue1").
|
|
|
getResourceUsage().getMemory());
|
|
|
|
|
|
- NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2,
|
|
|
- new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
|
|
|
scheduler.handle(updateEvent2);
|
|
|
|
|
|
assertEquals(1024, scheduler.getQueueManager().getQueue("queue1").
|
|
|
getResourceUsage().getMemory());
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test (timeout = 5000)
|
|
|
public void testSimpleContainerReservation() throws InterruptedException {
|
|
|
// Add a node
|
|
|
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
|
|
@@ -310,8 +307,7 @@ public class TestFairScheduler {
|
|
|
// Queue 1 requests full capacity of node
|
|
|
createSchedulingRequest(1024, "queue1", "user1", 1);
|
|
|
scheduler.update();
|
|
|
- NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1,
|
|
|
- new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
|
|
scheduler.handle(updateEvent);
|
|
|
|
|
|
// Make sure queue 1 is allocated app capacity
|
|
@@ -331,8 +327,7 @@ public class TestFairScheduler {
|
|
|
// Now another node checks in with capacity
|
|
|
RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
|
|
|
NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
|
|
|
- NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2,
|
|
|
- new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
|
|
|
scheduler.handle(nodeEvent2);
|
|
|
scheduler.handle(updateEvent2);
|
|
|
|
|
@@ -729,7 +724,7 @@ public class TestFairScheduler {
|
|
|
assertEquals(300000, queueManager.getFairSharePreemptionTimeout());
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test (timeout = 5000)
|
|
|
public void testIsStarvedForMinShare() throws Exception {
|
|
|
Configuration conf = createConfiguration();
|
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
@@ -758,8 +753,7 @@ public class TestFairScheduler {
|
|
|
// Queue A wants 3 * 1024. Node update gives this all to A
|
|
|
createSchedulingRequest(3 * 1024, "queueA", "user1");
|
|
|
scheduler.update();
|
|
|
- NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1,
|
|
|
- new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1);
|
|
|
scheduler.handle(nodeEvent2);
|
|
|
|
|
|
// Queue B arrives and wants 1 * 1024
|
|
@@ -788,7 +782,7 @@ public class TestFairScheduler {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test (timeout = 5000)
|
|
|
public void testIsStarvedForFairShare() throws Exception {
|
|
|
Configuration conf = createConfiguration();
|
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
@@ -817,8 +811,7 @@ public class TestFairScheduler {
|
|
|
// Queue A wants 3 * 1024. Node update gives this all to A
|
|
|
createSchedulingRequest(3 * 1024, "queueA", "user1");
|
|
|
scheduler.update();
|
|
|
- NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1,
|
|
|
- new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1);
|
|
|
scheduler.handle(nodeEvent2);
|
|
|
|
|
|
// Queue B arrives and wants 1 * 1024
|
|
@@ -848,7 +841,7 @@ public class TestFairScheduler {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test (timeout = 5000)
|
|
|
/**
|
|
|
* Make sure containers are chosen to be preempted in the correct order. Right
|
|
|
* now this means decreasing order of priority.
|
|
@@ -912,16 +905,13 @@ public class TestFairScheduler {
|
|
|
|
|
|
// Sufficient node check-ins to fully schedule containers
|
|
|
for (int i = 0; i < 2; i++) {
|
|
|
- NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1,
|
|
|
- new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1);
|
|
|
scheduler.handle(nodeUpdate1);
|
|
|
|
|
|
- NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2,
|
|
|
- new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2);
|
|
|
scheduler.handle(nodeUpdate2);
|
|
|
|
|
|
- NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3,
|
|
|
- new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3);
|
|
|
scheduler.handle(nodeUpdate3);
|
|
|
}
|
|
|
|
|
@@ -982,7 +972,7 @@ public class TestFairScheduler {
|
|
|
assertEquals(0, scheduler.applications.get(app6).getLiveContainers().size());
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test (timeout = 5000)
|
|
|
/**
|
|
|
* Tests the timing of decision to preempt tasks.
|
|
|
*/
|
|
@@ -1053,16 +1043,13 @@ public class TestFairScheduler {
|
|
|
|
|
|
// Sufficient node check-ins to fully schedule containers
|
|
|
for (int i = 0; i < 2; i++) {
|
|
|
- NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1,
|
|
|
- new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1);
|
|
|
scheduler.handle(nodeUpdate1);
|
|
|
|
|
|
- NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2,
|
|
|
- new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2);
|
|
|
scheduler.handle(nodeUpdate2);
|
|
|
|
|
|
- NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3,
|
|
|
- new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3);
|
|
|
scheduler.handle(nodeUpdate3);
|
|
|
}
|
|
|
|
|
@@ -1110,7 +1097,7 @@ public class TestFairScheduler {
|
|
|
Resources.createResource(1536), scheduler.resToPreempt(schedD, clock.getTime())));
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test (timeout = 5000)
|
|
|
public void testMultipleContainersWaitingForReservation() {
|
|
|
// Add a node
|
|
|
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
|
|
@@ -1120,8 +1107,7 @@ public class TestFairScheduler {
|
|
|
// Request full capacity of node
|
|
|
createSchedulingRequest(1024, "queue1", "user1", 1);
|
|
|
scheduler.update();
|
|
|
- NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1,
|
|
|
- new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
|
|
scheduler.handle(updateEvent);
|
|
|
|
|
|
ApplicationAttemptId attId1 = createSchedulingRequest(1024, "queue2", "user2", 1);
|
|
@@ -1137,7 +1123,7 @@ public class TestFairScheduler {
|
|
|
scheduler.applications.get(attId2).getCurrentReservation().getMemory());
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test (timeout = 5000)
|
|
|
public void testUserMaxRunningApps() throws Exception {
|
|
|
// Set max running apps
|
|
|
Configuration conf = createConfiguration();
|
|
@@ -1166,8 +1152,7 @@ public class TestFairScheduler {
|
|
|
"user1", 1);
|
|
|
|
|
|
scheduler.update();
|
|
|
- NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1,
|
|
|
- new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
|
|
scheduler.handle(updateEvent);
|
|
|
|
|
|
// App 1 should be running
|
|
@@ -1192,7 +1177,7 @@ public class TestFairScheduler {
|
|
|
assertEquals(2, scheduler.applications.get(attId1).getLiveContainers().size());
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test (timeout = 5000)
|
|
|
public void testReservationWhileMultiplePriorities() {
|
|
|
// Add a node
|
|
|
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
|
|
@@ -1202,8 +1187,7 @@ public class TestFairScheduler {
|
|
|
ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1",
|
|
|
"user1", 1, 2);
|
|
|
scheduler.update();
|
|
|
- NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1,
|
|
|
- new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
|
|
scheduler.handle(updateEvent);
|
|
|
|
|
|
FSSchedulerApp app = scheduler.applications.get(attId);
|
|
@@ -1276,7 +1260,7 @@ public class TestFairScheduler {
|
|
|
assertNull("The application was allowed", app2);
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test (timeout = 5000)
|
|
|
public void testMultipleNodesSingleRackRequest() throws Exception {
|
|
|
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
|
|
|
RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
|
|
@@ -1303,22 +1287,20 @@ public class TestFairScheduler {
|
|
|
|
|
|
// node 1 checks in
|
|
|
scheduler.update();
|
|
|
- NodeUpdateSchedulerEvent updateEvent1 = new NodeUpdateSchedulerEvent(node1,
|
|
|
- new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent updateEvent1 = new NodeUpdateSchedulerEvent(node1);
|
|
|
scheduler.handle(updateEvent1);
|
|
|
// should assign node local
|
|
|
assertEquals(1, scheduler.applications.get(appId).getLiveContainers().size());
|
|
|
|
|
|
// node 2 checks in
|
|
|
scheduler.update();
|
|
|
- NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2,
|
|
|
- new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
|
|
|
scheduler.handle(updateEvent2);
|
|
|
// should assign rack local
|
|
|
assertEquals(2, scheduler.applications.get(appId).getLiveContainers().size());
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test (timeout = 5000)
|
|
|
public void testFifoWithinQueue() throws Exception {
|
|
|
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(3072));
|
|
|
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
|
|
@@ -1342,8 +1324,7 @@ public class TestFairScheduler {
|
|
|
// Because tests set assignmultiple to false, each heartbeat assigns a single
|
|
|
// container.
|
|
|
|
|
|
- NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1,
|
|
|
- new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
|
|
|
+ NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
|
|
|
|
|
scheduler.handle(updateEvent);
|
|
|
assertEquals(1, app1.getLiveContainers().size());
|