|
@@ -68,6 +68,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
|
|
|
|
+
|
|
|
|
+
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
|
@@ -89,6 +92,8 @@ import org.junit.Test;
|
|
import org.mockito.Matchers;
|
|
import org.mockito.Matchers;
|
|
import org.mockito.Mockito;
|
|
import org.mockito.Mockito;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSchedulerKey;
|
|
|
|
+
|
|
public class TestLeafQueue {
|
|
public class TestLeafQueue {
|
|
private final RecordFactory recordFactory =
|
|
private final RecordFactory recordFactory =
|
|
RecordFactoryProvider.getRecordFactory(null);
|
|
RecordFactoryProvider.getRecordFactory(null);
|
|
@@ -731,6 +736,7 @@ public class TestLeafQueue {
|
|
qb.getActiveUsersManager(), spyRMContext);
|
|
qb.getActiveUsersManager(), spyRMContext);
|
|
qb.submitApplicationAttempt(app_0, user_0);
|
|
qb.submitApplicationAttempt(app_0, user_0);
|
|
Priority u0Priority = TestUtils.createMockPriority(1);
|
|
Priority u0Priority = TestUtils.createMockPriority(1);
|
|
|
|
+ SchedulerRequestKey u0SchedKey = toSchedulerKey(u0Priority);
|
|
app_0.updateResourceRequests(Collections.singletonList(
|
|
app_0.updateResourceRequests(Collections.singletonList(
|
|
TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true,
|
|
TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true,
|
|
u0Priority, recordFactory)));
|
|
u0Priority, recordFactory)));
|
|
@@ -753,6 +759,7 @@ public class TestLeafQueue {
|
|
new FiCaSchedulerApp(appAttemptId_2, user_1, qb,
|
|
new FiCaSchedulerApp(appAttemptId_2, user_1, qb,
|
|
qb.getActiveUsersManager(), spyRMContext);
|
|
qb.getActiveUsersManager(), spyRMContext);
|
|
Priority u1Priority = TestUtils.createMockPriority(2);
|
|
Priority u1Priority = TestUtils.createMockPriority(2);
|
|
|
|
+ SchedulerRequestKey u1SchedKey = toSchedulerKey(u1Priority);
|
|
app_2.updateResourceRequests(Collections.singletonList(
|
|
app_2.updateResourceRequests(Collections.singletonList(
|
|
TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true,
|
|
TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true,
|
|
u1Priority, recordFactory)));
|
|
u1Priority, recordFactory)));
|
|
@@ -773,9 +780,9 @@ public class TestLeafQueue {
|
|
//test case 3
|
|
//test case 3
|
|
qb.finishApplication(app_0.getApplicationId(), user_0);
|
|
qb.finishApplication(app_0.getApplicationId(), user_0);
|
|
qb.finishApplication(app_2.getApplicationId(), user_1);
|
|
qb.finishApplication(app_2.getApplicationId(), user_1);
|
|
- qb.releaseResource(clusterResource, app_0, app_0.getResource(u0Priority),
|
|
|
|
|
|
+ qb.releaseResource(clusterResource, app_0, app_0.getResource(u0SchedKey),
|
|
null, null, false);
|
|
null, null, false);
|
|
- qb.releaseResource(clusterResource, app_2, app_2.getResource(u1Priority),
|
|
|
|
|
|
+ qb.releaseResource(clusterResource, app_2, app_2.getResource(u1SchedKey),
|
|
null, null, false);
|
|
null, null, false);
|
|
|
|
|
|
qb.setUserLimit(50);
|
|
qb.setUserLimit(50);
|
|
@@ -1452,7 +1459,7 @@ public class TestLeafQueue {
|
|
assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
|
|
assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
|
|
assertEquals(4*GB, app_1.getCurrentReservation().getMemorySize());
|
|
assertEquals(4*GB, app_1.getCurrentReservation().getMemorySize());
|
|
assertEquals(1*GB, node_0.getAllocatedResource().getMemorySize());
|
|
assertEquals(1*GB, node_0.getAllocatedResource().getMemorySize());
|
|
- assertEquals(1, app_1.getReReservations(priority));
|
|
|
|
|
|
+ assertEquals(1, app_1.getReReservations(toSchedulerKey(priority)));
|
|
|
|
|
|
// Re-reserve
|
|
// Re-reserve
|
|
a.assignContainers(clusterResource, node_0,
|
|
a.assignContainers(clusterResource, node_0,
|
|
@@ -1462,7 +1469,7 @@ public class TestLeafQueue {
|
|
assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
|
|
assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
|
|
assertEquals(4*GB, app_1.getCurrentReservation().getMemorySize());
|
|
assertEquals(4*GB, app_1.getCurrentReservation().getMemorySize());
|
|
assertEquals(1*GB, node_0.getAllocatedResource().getMemorySize());
|
|
assertEquals(1*GB, node_0.getAllocatedResource().getMemorySize());
|
|
- assertEquals(2, app_1.getReReservations(priority));
|
|
|
|
|
|
+ assertEquals(2, app_1.getReReservations(toSchedulerKey(priority)));
|
|
|
|
|
|
// Try to schedule on node_1 now, should *move* the reservation
|
|
// Try to schedule on node_1 now, should *move* the reservation
|
|
a.assignContainers(clusterResource, node_1,
|
|
a.assignContainers(clusterResource, node_1,
|
|
@@ -1474,7 +1481,7 @@ public class TestLeafQueue {
|
|
assertEquals(4*GB, node_1.getAllocatedResource().getMemorySize());
|
|
assertEquals(4*GB, node_1.getAllocatedResource().getMemorySize());
|
|
// Doesn't change yet... only when reservation is cancelled or a different
|
|
// Doesn't change yet... only when reservation is cancelled or a different
|
|
// container is reserved
|
|
// container is reserved
|
|
- assertEquals(2, app_1.getReReservations(priority));
|
|
|
|
|
|
+ assertEquals(2, app_1.getReReservations(toSchedulerKey(priority)));
|
|
|
|
|
|
// Now finish another container from app_0 and see the reservation cancelled
|
|
// Now finish another container from app_0 and see the reservation cancelled
|
|
rmContainer = app_0.getLiveContainers().iterator().next();
|
|
rmContainer = app_0.getLiveContainers().iterator().next();
|
|
@@ -1564,29 +1571,30 @@ public class TestLeafQueue {
|
|
|
|
|
|
// Start testing...
|
|
// Start testing...
|
|
CSAssignment assignment = null;
|
|
CSAssignment assignment = null;
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ SchedulerRequestKey schedulerKey = toSchedulerKey(priority);
|
|
// Start with off switch, shouldn't allocate due to delay scheduling
|
|
// Start with off switch, shouldn't allocate due to delay scheduling
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(1, app_0.getSchedulingOpportunities(priority));
|
|
|
|
- assertEquals(3, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
|
|
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
|
|
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
|
|
|
|
|
|
// Another off switch, shouldn't allocate due to delay scheduling
|
|
// Another off switch, shouldn't allocate due to delay scheduling
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(2, app_0.getSchedulingOpportunities(priority));
|
|
|
|
- assertEquals(3, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
|
|
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
|
|
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
|
|
|
|
|
|
// Another off switch, shouldn't allocate due to delay scheduling
|
|
// Another off switch, shouldn't allocate due to delay scheduling
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(3, app_0.getSchedulingOpportunities(priority));
|
|
|
|
- assertEquals(3, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
|
|
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
|
|
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
|
|
|
|
|
|
// Another off switch, now we should allocate
|
|
// Another off switch, now we should allocate
|
|
@@ -1594,22 +1602,25 @@ public class TestLeafQueue {
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
|
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
|
- assertEquals(4, app_0.getSchedulingOpportunities(priority)); // should NOT reset
|
|
|
|
- assertEquals(2, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ // should NOT reset
|
|
|
|
+ assertEquals(4, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(2, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
// NODE_LOCAL - node_0
|
|
// NODE_LOCAL - node_0
|
|
assignment = a.assignContainers(clusterResource, node_0,
|
|
assignment = a.assignContainers(clusterResource, node_0,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
|
|
|
|
- assertEquals(1, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ // should reset
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(1, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
// NODE_LOCAL - node_1
|
|
// NODE_LOCAL - node_1
|
|
assignment = a.assignContainers(clusterResource, node_1,
|
|
assignment = a.assignContainers(clusterResource, node_1,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
|
|
|
|
- assertEquals(0, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ // should reset
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
|
|
assertEquals(NodeType.NODE_LOCAL, assignment.getType());
|
|
assertEquals(NodeType.NODE_LOCAL, assignment.getType());
|
|
|
|
|
|
// Add 1 more request to check for RACK_LOCAL
|
|
// Add 1 more request to check for RACK_LOCAL
|
|
@@ -1624,7 +1635,7 @@ public class TestLeafQueue {
|
|
TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 4, // one extra
|
|
TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 4, // one extra
|
|
true, priority, recordFactory));
|
|
true, priority, recordFactory));
|
|
app_0.updateResourceRequests(app_0_requests_0);
|
|
app_0.updateResourceRequests(app_0_requests_0);
|
|
- assertEquals(4, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ assertEquals(4, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
String host_3 = "127.0.0.4"; // on rack_1
|
|
String host_3 = "127.0.0.4"; // on rack_1
|
|
FiCaSchedulerNode node_3 = TestUtils.getMockNode(host_3, rack_1, 0, 8*GB);
|
|
FiCaSchedulerNode node_3 = TestUtils.getMockNode(host_3, rack_1, 0, 8*GB);
|
|
@@ -1636,21 +1647,22 @@ public class TestLeafQueue {
|
|
// Shouldn't assign RACK_LOCAL yet
|
|
// Shouldn't assign RACK_LOCAL yet
|
|
assignment = a.assignContainers(clusterResource, node_3,
|
|
assignment = a.assignContainers(clusterResource, node_3,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
- assertEquals(1, app_0.getSchedulingOpportunities(priority));
|
|
|
|
- assertEquals(4, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(4, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
// Should assign RACK_LOCAL now
|
|
// Should assign RACK_LOCAL now
|
|
assignment = a.assignContainers(clusterResource, node_3,
|
|
assignment = a.assignContainers(clusterResource, node_3,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
|
|
verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
|
|
|
|
- assertEquals(3, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ // should reset
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
// Shouldn't assign RACK_LOCAL because schedulingOpportunities should have gotten reset.
|
|
// Shouldn't assign RACK_LOCAL because schedulingOpportunities should have gotten reset.
|
|
assignment = a.assignContainers(clusterResource, node_3,
|
|
assignment = a.assignContainers(clusterResource, node_3,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
- assertEquals(1, app_0.getSchedulingOpportunities(priority));
|
|
|
|
- assertEquals(3, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
// Next time we schedule RACK_LOCAL, don't reset
|
|
// Next time we schedule RACK_LOCAL, don't reset
|
|
doReturn(false).when(a).getRackLocalityFullReset();
|
|
doReturn(false).when(a).getRackLocalityFullReset();
|
|
@@ -1659,19 +1671,21 @@ public class TestLeafQueue {
|
|
assignment = a.assignContainers(clusterResource, node_3,
|
|
assignment = a.assignContainers(clusterResource, node_3,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
|
|
verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
|
|
- assertEquals(2, app_0.getSchedulingOpportunities(priority)); // should NOT reset
|
|
|
|
- assertEquals(2, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ // should NOT reset
|
|
|
|
+ assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(2, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
// Another RACK_LOCAL since schedulingOpportunities not reset
|
|
// Another RACK_LOCAL since schedulingOpportunities not reset
|
|
assignment = a.assignContainers(clusterResource, node_3,
|
|
assignment = a.assignContainers(clusterResource, node_3,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
|
|
verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
|
|
- assertEquals(3, app_0.getSchedulingOpportunities(priority)); // should NOT reset
|
|
|
|
- assertEquals(1, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ // should NOT reset
|
|
|
|
+ assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(1, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
// Add a request larger than cluster size to verify
|
|
// Add a request larger than cluster size to verify
|
|
// OFF_SWITCH delay is capped by cluster size
|
|
// OFF_SWITCH delay is capped by cluster size
|
|
- app_0.resetSchedulingOpportunities(priority);
|
|
|
|
|
|
+ app_0.resetSchedulingOpportunities(schedulerKey);
|
|
app_0_requests_0.clear();
|
|
app_0_requests_0.clear();
|
|
app_0_requests_0.add(
|
|
app_0_requests_0.add(
|
|
TestUtils.createResourceRequest(host_0, 1*GB, 100,
|
|
TestUtils.createResourceRequest(host_0, 1*GB, 100,
|
|
@@ -1690,13 +1704,13 @@ public class TestLeafQueue {
|
|
a.assignContainers(clusterResource, node_2, new ResourceLimits(
|
|
a.assignContainers(clusterResource, node_2, new ResourceLimits(
|
|
clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(i+1, app_0.getSchedulingOpportunities(priority));
|
|
|
|
|
|
+ assertEquals(i+1, app_0.getSchedulingOpportunities(schedulerKey));
|
|
}
|
|
}
|
|
// delay should be capped at numNodes so next one should allocate
|
|
// delay should be capped at numNodes so next one should allocate
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
|
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
|
- assertEquals(numNodes+1, app_0.getSchedulingOpportunities(priority));
|
|
|
|
|
|
+ assertEquals(numNodes+1, app_0.getSchedulingOpportunities(schedulerKey));
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
@@ -1738,6 +1752,7 @@ public class TestLeafQueue {
|
|
|
|
|
|
// P1
|
|
// P1
|
|
Priority priority_1 = TestUtils.createMockPriority(1);
|
|
Priority priority_1 = TestUtils.createMockPriority(1);
|
|
|
|
+ SchedulerRequestKey schedulerKey1 = toSchedulerKey(priority_1);
|
|
app_0_requests_0.add(
|
|
app_0_requests_0.add(
|
|
TestUtils.createResourceRequest(host_0, 1*GB, 1,
|
|
TestUtils.createResourceRequest(host_0, 1*GB, 1,
|
|
true, priority_1, recordFactory));
|
|
true, priority_1, recordFactory));
|
|
@@ -1756,6 +1771,7 @@ public class TestLeafQueue {
|
|
|
|
|
|
// P2
|
|
// P2
|
|
Priority priority_2 = TestUtils.createMockPriority(2);
|
|
Priority priority_2 = TestUtils.createMockPriority(2);
|
|
|
|
+ SchedulerRequestKey schedulerKey2 = toSchedulerKey(priority_2);
|
|
app_0_requests_0.add(
|
|
app_0_requests_0.add(
|
|
TestUtils.createResourceRequest(host_2, 2*GB, 1,
|
|
TestUtils.createResourceRequest(host_2, 2*GB, 1,
|
|
true, priority_2, recordFactory));
|
|
true, priority_2, recordFactory));
|
|
@@ -1775,47 +1791,47 @@ public class TestLeafQueue {
|
|
CSAssignment assignment = a.assignContainers(clusterResource, node_2,
|
|
CSAssignment assignment = a.assignContainers(clusterResource, node_2,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(1, app_0.getSchedulingOpportunities(priority_1));
|
|
|
|
- assertEquals(2, app_0.getTotalRequiredResources(priority_1));
|
|
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority_2));
|
|
|
|
- assertEquals(1, app_0.getTotalRequiredResources(priority_2));
|
|
|
|
|
|
+ assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey1));
|
|
|
|
+ assertEquals(2, app_0.getTotalRequiredResources(schedulerKey1));
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
|
|
|
|
+ assertEquals(1, app_0.getTotalRequiredResources(schedulerKey2));
|
|
|
|
|
|
// Another off-switch, shouldn't allocate P1 due to delay scheduling
|
|
// Another off-switch, shouldn't allocate P1 due to delay scheduling
|
|
// thus, no P2 either!
|
|
// thus, no P2 either!
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(2, app_0.getSchedulingOpportunities(priority_1));
|
|
|
|
- assertEquals(2, app_0.getTotalRequiredResources(priority_1));
|
|
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority_2));
|
|
|
|
- assertEquals(1, app_0.getTotalRequiredResources(priority_2));
|
|
|
|
|
|
+ assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey1));
|
|
|
|
+ assertEquals(2, app_0.getTotalRequiredResources(schedulerKey1));
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
|
|
|
|
+ assertEquals(1, app_0.getTotalRequiredResources(schedulerKey2));
|
|
|
|
|
|
// Another off-switch, shouldn't allocate OFF_SWITCH P1
|
|
// Another off-switch, shouldn't allocate OFF_SWITCH P1
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
|
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
|
- assertEquals(3, app_0.getSchedulingOpportunities(priority_1));
|
|
|
|
- assertEquals(1, app_0.getTotalRequiredResources(priority_1));
|
|
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority_2));
|
|
|
|
- assertEquals(1, app_0.getTotalRequiredResources(priority_2));
|
|
|
|
|
|
+ assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey1));
|
|
|
|
+ assertEquals(1, app_0.getTotalRequiredResources(schedulerKey1));
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
|
|
|
|
+ assertEquals(1, app_0.getTotalRequiredResources(schedulerKey2));
|
|
|
|
|
|
// Now, DATA_LOCAL for P1
|
|
// Now, DATA_LOCAL for P1
|
|
assignment = a.assignContainers(clusterResource, node_0,
|
|
assignment = a.assignContainers(clusterResource, node_0,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority_1));
|
|
|
|
- assertEquals(0, app_0.getTotalRequiredResources(priority_1));
|
|
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority_2));
|
|
|
|
- assertEquals(1, app_0.getTotalRequiredResources(priority_2));
|
|
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey1));
|
|
|
|
+ assertEquals(0, app_0.getTotalRequiredResources(schedulerKey1));
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
|
|
|
|
+ assertEquals(1, app_0.getTotalRequiredResources(schedulerKey2));
|
|
|
|
|
|
// Now, OFF_SWITCH for P2
|
|
// Now, OFF_SWITCH for P2
|
|
assignment = a.assignContainers(clusterResource, node_1,
|
|
assignment = a.assignContainers(clusterResource, node_1,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
|
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority_1));
|
|
|
|
- assertEquals(0, app_0.getTotalRequiredResources(priority_1));
|
|
|
|
- assertEquals(1, app_0.getSchedulingOpportunities(priority_2));
|
|
|
|
- assertEquals(0, app_0.getTotalRequiredResources(priority_2));
|
|
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey1));
|
|
|
|
+ assertEquals(0, app_0.getTotalRequiredResources(schedulerKey1));
|
|
|
|
+ assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey2));
|
|
|
|
+ assertEquals(0, app_0.getTotalRequiredResources(schedulerKey2));
|
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1855,6 +1871,7 @@ public class TestLeafQueue {
|
|
|
|
|
|
// Setup resource-requests and submit
|
|
// Setup resource-requests and submit
|
|
Priority priority = TestUtils.createMockPriority(1);
|
|
Priority priority = TestUtils.createMockPriority(1);
|
|
|
|
+ SchedulerRequestKey schedulerKey = toSchedulerKey(priority);
|
|
List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
|
|
List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
|
|
app_0_requests_0.add(
|
|
app_0_requests_0.add(
|
|
TestUtils.createResourceRequest(host_0_0, 1*GB, 1,
|
|
TestUtils.createResourceRequest(host_0_0, 1*GB, 1,
|
|
@@ -1878,7 +1895,7 @@ public class TestLeafQueue {
|
|
// Add one request
|
|
// Add one request
|
|
app_0_requests_0.clear();
|
|
app_0_requests_0.clear();
|
|
app_0_requests_0.add(
|
|
app_0_requests_0.add(
|
|
- TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, // only one
|
|
|
|
|
|
+ TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, // only 1
|
|
true, priority, recordFactory));
|
|
true, priority, recordFactory));
|
|
app_0.updateResourceRequests(app_0_requests_0);
|
|
app_0.updateResourceRequests(app_0_requests_0);
|
|
|
|
|
|
@@ -1886,17 +1903,19 @@ public class TestLeafQueue {
|
|
CSAssignment assignment = a.assignContainers(clusterResource, node_0_0,
|
|
CSAssignment assignment = a.assignContainers(clusterResource, node_0_0,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
|
|
|
|
- assertEquals(0, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ // should reset
|
|
|
|
+ assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
// No allocation on node_1_0 even though it's node/rack local since
|
|
// No allocation on node_1_0 even though it's node/rack local since
|
|
// required(ANY) == 0
|
|
// required(ANY) == 0
|
|
assignment = a.assignContainers(clusterResource, node_1_0,
|
|
assignment = a.assignContainers(clusterResource, node_1_0,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority)); // Still zero
|
|
|
|
- // since #req=0
|
|
|
|
- assertEquals(0, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ // Still zero
|
|
|
|
+ // since #req=0
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
// Add one request
|
|
// Add one request
|
|
app_0_requests_0.clear();
|
|
app_0_requests_0.clear();
|
|
@@ -1910,15 +1929,16 @@ public class TestLeafQueue {
|
|
assignment = a.assignContainers(clusterResource, node_0_1,
|
|
assignment = a.assignContainers(clusterResource, node_0_1,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(1, app_0.getSchedulingOpportunities(priority));
|
|
|
|
- assertEquals(1, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(1, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
// NODE_LOCAL - node_1
|
|
// NODE_LOCAL - node_1
|
|
assignment = a.assignContainers(clusterResource, node_1_0,
|
|
assignment = a.assignContainers(clusterResource, node_1_0,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
|
|
|
|
- assertEquals(0, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ // should reset
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
|
|
}
|
|
}
|
|
|
|
|
|
@Test (timeout = 30000)
|
|
@Test (timeout = 30000)
|
|
@@ -2142,6 +2162,7 @@ public class TestLeafQueue {
|
|
// host_1_1: 8G
|
|
// host_1_1: 8G
|
|
// Blacklist: <host_0_0>
|
|
// Blacklist: <host_0_0>
|
|
Priority priority = TestUtils.createMockPriority(1);
|
|
Priority priority = TestUtils.createMockPriority(1);
|
|
|
|
+ SchedulerRequestKey schedulerKey = toSchedulerKey(priority);
|
|
List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
|
|
List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
|
|
app_0_requests_0.add(
|
|
app_0_requests_0.add(
|
|
TestUtils.createResourceRequest(host_0_0, 1*GB, 1,
|
|
TestUtils.createResourceRequest(host_0_0, 1*GB, 1,
|
|
@@ -2169,7 +2190,8 @@ public class TestLeafQueue {
|
|
a.assignContainers(clusterResource, node_0_1, new ResourceLimits(
|
|
a.assignContainers(clusterResource, node_0_1, new ResourceLimits(
|
|
clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
|
|
|
|
|
|
+ // should be 0
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
|
|
// resourceName: <priority, memory, #containers, relaxLocality>
|
|
// resourceName: <priority, memory, #containers, relaxLocality>
|
|
// host_0_0: < 1, 1GB, 1, true >
|
|
// host_0_0: < 1, 1GB, 1, true >
|
|
@@ -2191,7 +2213,8 @@ public class TestLeafQueue {
|
|
assignment = a.assignContainers(clusterResource, node_1_1,
|
|
assignment = a.assignContainers(clusterResource, node_1_1,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
|
|
|
|
|
|
+ // should be 0
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
|
|
// Allow rack-locality for rack_1, but blacklist node_1_1
|
|
// Allow rack-locality for rack_1, but blacklist node_1_1
|
|
app_0_requests_0.add(
|
|
app_0_requests_0.add(
|
|
@@ -2221,7 +2244,8 @@ public class TestLeafQueue {
|
|
assignment = a.assignContainers(clusterResource, node_1_1,
|
|
assignment = a.assignContainers(clusterResource, node_1_1,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
|
|
|
|
|
|
+ // should be 0
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
|
|
// Now, remove node_1_1 from blacklist, but add rack_1 to blacklist
|
|
// Now, remove node_1_1 from blacklist, but add rack_1 to blacklist
|
|
app_0.updateResourceRequests(app_0_requests_0);
|
|
app_0.updateResourceRequests(app_0_requests_0);
|
|
@@ -2249,7 +2273,8 @@ public class TestLeafQueue {
|
|
assignment = a.assignContainers(clusterResource, node_1_1,
|
|
assignment = a.assignContainers(clusterResource, node_1_1,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
|
|
|
|
|
|
+ // should be 0
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
|
|
// Now remove rack_1 from blacklist
|
|
// Now remove rack_1 from blacklist
|
|
app_0.updateResourceRequests(app_0_requests_0);
|
|
app_0.updateResourceRequests(app_0_requests_0);
|
|
@@ -2275,8 +2300,8 @@ public class TestLeafQueue {
|
|
assignment = a.assignContainers(clusterResource, node_1_1,
|
|
assignment = a.assignContainers(clusterResource, node_1_1,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyNoContainerAllocated(assignment);
|
|
verifyNoContainerAllocated(assignment);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority));
|
|
|
|
- assertEquals(1, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(1, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
// Now sanity-check node_local
|
|
// Now sanity-check node_local
|
|
app_0_requests_0.add(
|
|
app_0_requests_0.add(
|
|
@@ -2305,8 +2330,8 @@ public class TestLeafQueue {
|
|
assignment = a.assignContainers(clusterResource, node_1_0,
|
|
assignment = a.assignContainers(clusterResource, node_1_0,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
- assertEquals(0, app_0.getSchedulingOpportunities(priority));
|
|
|
|
- assertEquals(0, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
|
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
@@ -2667,6 +2692,7 @@ public class TestLeafQueue {
|
|
// App0 has node local request for host_0/host_1, and app1 has node local
|
|
// App0 has node local request for host_0/host_1, and app1 has node local
|
|
// request for host2.
|
|
// request for host2.
|
|
Priority priority = TestUtils.createMockPriority(1);
|
|
Priority priority = TestUtils.createMockPriority(1);
|
|
|
|
+ SchedulerRequestKey schedulerKey = toSchedulerKey(priority);
|
|
List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
|
|
List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
|
|
app_0_requests_0.add(
|
|
app_0_requests_0.add(
|
|
TestUtils.createResourceRequest(host_0, 1*GB, 1,
|
|
TestUtils.createResourceRequest(host_0, 1*GB, 1,
|
|
@@ -2706,8 +2732,8 @@ public class TestLeafQueue {
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
assignment = a.assignContainers(clusterResource, node_2,
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
|
- assertEquals(1, app_0.getSchedulingOpportunities(priority));
|
|
|
|
- assertEquals(3, app_0.getTotalRequiredResources(priority));
|
|
|
|
|
|
+ assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
|
|
|
+ assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
|
|
assertEquals(0, app_0.getLiveContainers().size());
|
|
assertEquals(0, app_0.getLiveContainers().size());
|
|
assertEquals(1, app_1.getLiveContainers().size());
|
|
assertEquals(1, app_1.getLiveContainers().size());
|
|
}
|
|
}
|