|
@@ -18,13 +18,17 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.reservation.planning;
|
|
|
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
import java.util.Random;
|
|
|
|
|
|
import org.apache.hadoop.yarn.api.records.ReservationDefinition;
|
|
@@ -37,29 +41,32 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.reservation.CapacityOverTimePolicy;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryPlan;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryReservationAllocation;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationAllocation;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInterval;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystemTestUtil;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.AlignedPlannerWithGreedy;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.StageAllocatorLowCostAligned.DurationInterval;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
|
|
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
+import org.eclipse.jetty.util.log.Log;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
-import org.mortbay.log.Log;
|
|
|
|
|
|
public class TestAlignedPlanner {
|
|
|
|
|
|
- ReservationAgent agent;
|
|
|
- InMemoryPlan plan;
|
|
|
- Resource minAlloc = Resource.newInstance(1024, 1);
|
|
|
- ResourceCalculator res = new DefaultResourceCalculator();
|
|
|
- Resource maxAlloc = Resource.newInstance(1024 * 8, 8);
|
|
|
- Random rand = new Random();
|
|
|
- long step;
|
|
|
+ private ReservationAgent agentRight;
|
|
|
+ private ReservationAgent agentLeft;
|
|
|
+ private InMemoryPlan plan;
|
|
|
+ private final Resource minAlloc = Resource.newInstance(1024, 1);
|
|
|
+ private final ResourceCalculator res = new DefaultResourceCalculator();
|
|
|
+ private final Resource maxAlloc = Resource.newInstance(1024 * 8, 8);
|
|
|
+ private final Random rand = new Random();
|
|
|
+ private Resource clusterCapacity;
|
|
|
+ private long step;
|
|
|
|
|
|
@Test
|
|
|
public void testSingleReservationAccept() throws PlanningException {
|
|
@@ -82,7 +89,7 @@ public class TestAlignedPlanner {
|
|
|
// Add reservation
|
|
|
ReservationId reservationID =
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
- agent.createReservation(reservationID, "u1", plan, rr1);
|
|
|
+ agentRight.createReservation(reservationID, "u1", plan, rr1);
|
|
|
|
|
|
// CHECK: allocation was accepted
|
|
|
assertTrue("Agent-based allocation failed", reservationID != null);
|
|
@@ -107,7 +114,7 @@ public class TestAlignedPlanner {
|
|
|
// Create reservation
|
|
|
ReservationDefinition rr1 =
|
|
|
createReservationDefinition(
|
|
|
- 10L, // Job arrival time
|
|
|
+ 10 * step, // Job arrival time
|
|
|
15 * step, // Job deadline
|
|
|
new ReservationRequest[] {
|
|
|
ReservationRequest.newInstance(
|
|
@@ -126,7 +133,7 @@ public class TestAlignedPlanner {
|
|
|
try {
|
|
|
ReservationId reservationID =
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
- agent.createReservation(reservationID, "u1", plan, rr1);
|
|
|
+ agentRight.createReservation(reservationID, "u1", plan, rr1);
|
|
|
fail();
|
|
|
} catch (PlanningException e) {
|
|
|
// Expected failure
|
|
@@ -166,7 +173,7 @@ public class TestAlignedPlanner {
|
|
|
try {
|
|
|
ReservationId reservationID =
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
- agent.createReservation(reservationID, "u1", plan, rr1);
|
|
|
+ agentRight.createReservation(reservationID, "u1", plan, rr1);
|
|
|
fail();
|
|
|
} catch (PlanningException e) {
|
|
|
// Expected failure
|
|
@@ -206,7 +213,7 @@ public class TestAlignedPlanner {
|
|
|
try {
|
|
|
ReservationId reservationID =
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
- agent.createReservation(reservationID, "u1", plan, rr1);
|
|
|
+ agentRight.createReservation(reservationID, "u1", plan, rr1);
|
|
|
fail();
|
|
|
} catch (PlanningException e) {
|
|
|
// Expected failure
|
|
@@ -246,7 +253,7 @@ public class TestAlignedPlanner {
|
|
|
try {
|
|
|
ReservationId reservationID =
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
- agent.createReservation(reservationID, "u1", plan, rr1);
|
|
|
+ agentRight.createReservation(reservationID, "u1", plan, rr1);
|
|
|
fail();
|
|
|
} catch (PlanningException e) {
|
|
|
// Expected failure
|
|
@@ -285,7 +292,7 @@ public class TestAlignedPlanner {
|
|
|
// Add reservation
|
|
|
ReservationId reservationID =
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
- agent.createReservation(reservationID, "u1", plan, rr1);
|
|
|
+ agentRight.createReservation(reservationID, "u1", plan, rr1);
|
|
|
|
|
|
// CHECK: allocation was accepted
|
|
|
assertTrue("Agent-based allocation failed", reservationID != null);
|
|
@@ -328,7 +335,7 @@ public class TestAlignedPlanner {
|
|
|
// Add reservation
|
|
|
ReservationId reservationID =
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
- agent.createReservation(reservationID, "u1", plan, rr1);
|
|
|
+ agentRight.createReservation(reservationID, "u1", plan, rr1);
|
|
|
|
|
|
// CHECK: allocation was accepted
|
|
|
assertTrue("Agent-based allocation failed", reservationID != null);
|
|
@@ -374,7 +381,7 @@ public class TestAlignedPlanner {
|
|
|
try {
|
|
|
ReservationId reservationID =
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
- agent.createReservation(reservationID, "u1", plan, rr1);
|
|
|
+ agentRight.createReservation(reservationID, "u1", plan, rr1);
|
|
|
fail();
|
|
|
} catch (PlanningException e) {
|
|
|
// Expected failure
|
|
@@ -420,10 +427,10 @@ public class TestAlignedPlanner {
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
|
|
|
// Add block, add flex, remove block, update flex
|
|
|
- agent.createReservation(blockReservationID, "uBlock", plan, rrBlock);
|
|
|
- agent.createReservation(flexReservationID, "uFlex", plan, rrFlex);
|
|
|
- agent.deleteReservation(blockReservationID, "uBlock", plan);
|
|
|
- agent.updateReservation(flexReservationID, "uFlex", plan, rrFlex);
|
|
|
+ agentRight.createReservation(blockReservationID, "uBlock", plan, rrBlock);
|
|
|
+ agentRight.createReservation(flexReservationID, "uFlex", plan, rrFlex);
|
|
|
+ agentRight.deleteReservation(blockReservationID, "uBlock", plan);
|
|
|
+ agentRight.updateReservation(flexReservationID, "uFlex", plan, rrFlex);
|
|
|
|
|
|
// CHECK: allocation was accepted
|
|
|
assertTrue("Agent-based allocation failed", flexReservationID != null);
|
|
@@ -458,7 +465,7 @@ public class TestAlignedPlanner {
|
|
|
try {
|
|
|
ReservationId reservationID =
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
- agent.createReservation(reservationID, "u1", plan, rr1);
|
|
|
+ agentRight.createReservation(reservationID, "u1", plan, rr1);
|
|
|
fail();
|
|
|
} catch (PlanningException e) {
|
|
|
// Expected failure
|
|
@@ -490,7 +497,7 @@ public class TestAlignedPlanner {
|
|
|
// Add reservation
|
|
|
ReservationId reservationID =
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
- agent.createReservation(reservationID, "u1", plan, rr1);
|
|
|
+ agentRight.createReservation(reservationID, "u1", plan, rr1);
|
|
|
|
|
|
// CHECK: allocation was accepted
|
|
|
assertTrue("Agent-based allocation failed", reservationID != null);
|
|
@@ -557,9 +564,9 @@ public class TestAlignedPlanner {
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
|
|
|
// Add all
|
|
|
- agent.createReservation(reservationID1, "u1", plan, rr7Mem1Core);
|
|
|
- agent.createReservation(reservationID2, "u2", plan, rr6Mem6Cores);
|
|
|
- agent.createReservation(reservationID3, "u3", plan, rr);
|
|
|
+ agentRight.createReservation(reservationID1, "u1", plan, rr7Mem1Core);
|
|
|
+ agentRight.createReservation(reservationID2, "u2", plan, rr6Mem6Cores);
|
|
|
+ agentRight.createReservation(reservationID3, "u3", plan, rr);
|
|
|
|
|
|
// Get reservation
|
|
|
ReservationAllocation alloc3 = plan.getReservationById(reservationID3);
|
|
@@ -684,8 +691,8 @@ public class TestAlignedPlanner {
|
|
|
for (ReservationDefinition rr : list) {
|
|
|
ReservationId reservationID =
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
- agent.createReservation(reservationID, "u" + Integer.toString(i), plan,
|
|
|
- rr);
|
|
|
+ agentRight.createReservation(reservationID, "u" + Integer.toString(i),
|
|
|
+ plan, rr);
|
|
|
++i;
|
|
|
}
|
|
|
|
|
@@ -695,13 +702,335 @@ public class TestAlignedPlanner {
|
|
|
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testSingleReservationAcceptAllocateLeft()
|
|
|
+ throws PlanningException {
|
|
|
+
|
|
|
+ // Create reservation
|
|
|
+ ReservationDefinition rr1 =
|
|
|
+ createReservationDefinition(
|
|
|
+ 10 * step, // Job arrival time
|
|
|
+ 35 * step, // Job deadline
|
|
|
+ new ReservationRequest[] {
|
|
|
+ ReservationRequest.newInstance(
|
|
|
+ Resource.newInstance(1024, 1), // Capability
|
|
|
+ 20, // Num containers
|
|
|
+ 20, // Concurrency
|
|
|
+ 10 * step), // Duration
|
|
|
+ ReservationRequest.newInstance(
|
|
|
+ Resource.newInstance(1024, 1), // Capability
|
|
|
+ 20, // Num containers
|
|
|
+ 20, // Concurrency
|
|
|
+ 10 * step) }, // Duration
|
|
|
+ ReservationRequestInterpreter.R_ORDER, "u1");
|
|
|
+
|
|
|
+ // Add reservation
|
|
|
+ ReservationId reservationID =
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ agentLeft.createReservation(reservationID, "u1", plan, rr1);
|
|
|
+
|
|
|
+ // CHECK: allocation was accepted
|
|
|
+ assertTrue("Agent-based allocation failed", reservationID != null);
|
|
|
+ assertTrue("Agent-based allocation failed", plan.getAllReservations()
|
|
|
+ .size() == 1);
|
|
|
+
|
|
|
+ // Get reservation
|
|
|
+ ReservationAllocation alloc1 = plan.getReservationById(reservationID);
|
|
|
+
|
|
|
+ // Verify allocation
|
|
|
+ assertTrue(alloc1.toString(),
|
|
|
+ check(alloc1, 10 * step, 30 * step, 20, 1024, 1));
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testLeftSucceedsRightFails() throws PlanningException {
|
|
|
+
|
|
|
+ // Prepare basic plan
|
|
|
+ int numJobsInScenario = initializeScenario2();
|
|
|
+
|
|
|
+ // Create reservation
|
|
|
+ ReservationDefinition rr1 =
|
|
|
+ createReservationDefinition(
|
|
|
+ 7 * step, // Job arrival time
|
|
|
+ 16 * step, // Job deadline
|
|
|
+ new ReservationRequest[] {
|
|
|
+ ReservationRequest.newInstance(Resource.newInstance(1024, 1),
|
|
|
+ 20, // Num containers
|
|
|
+ 20, // Concurrency
|
|
|
+ 2 * step), // Duration
|
|
|
+ ReservationRequest.newInstance(Resource.newInstance(1024, 1),
|
|
|
+ 20, // Num containers
|
|
|
+ 20, // Concurrency
|
|
|
+ 2 * step) }, // Duration
|
|
|
+ ReservationRequestInterpreter.R_ORDER, "u1");
|
|
|
+
|
|
|
+ ReservationDefinition rr2 =
|
|
|
+ createReservationDefinition(
|
|
|
+ 14 * step, // Job arrival time
|
|
|
+ 16 * step, // Job deadline
|
|
|
+ new ReservationRequest[] {
|
|
|
+ ReservationRequest.newInstance(
|
|
|
+ Resource.newInstance(1024, 1), // Capability
|
|
|
+ 100, // Num containers
|
|
|
+ 100, // Concurrency
|
|
|
+ 2 * step) }, // Duration
|
|
|
+ ReservationRequestInterpreter.R_ORDER, "u2");
|
|
|
+
|
|
|
+ // Add 1st reservation
|
|
|
+ ReservationId reservationID1 =
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ agentLeft.createReservation(reservationID1, "u1", plan, rr1);
|
|
|
+
|
|
|
+ // CHECK: allocation was accepted
|
|
|
+ assertTrue("Agent-based allocation failed", reservationID1 != null);
|
|
|
+ assertTrue("Agent-based allocation failed", plan.getAllReservations()
|
|
|
+ .size() == numJobsInScenario + 1);
|
|
|
+
|
|
|
+ // Get reservation
|
|
|
+ ReservationAllocation alloc1 = plan.getReservationById(reservationID1);
|
|
|
+
|
|
|
+ // Verify allocation
|
|
|
+ assertTrue(alloc1.toString(),
|
|
|
+ check(alloc1, 7 * step, 11 * step, 20, 1024, 1));
|
|
|
+
|
|
|
+ // Add second reservation
|
|
|
+ ReservationId reservationID2 =
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ agentLeft.createReservation(reservationID2, "u2", plan, rr2);
|
|
|
+
|
|
|
+ // CHECK: allocation was accepted
|
|
|
+ assertTrue("Agent-based allocation failed", reservationID2 != null);
|
|
|
+ assertTrue("Agent-based allocation failed", plan.getAllReservations()
|
|
|
+ .size() == numJobsInScenario + 2);
|
|
|
+
|
|
|
+ // Get reservation
|
|
|
+ ReservationAllocation alloc2 = plan.getReservationById(reservationID2);
|
|
|
+
|
|
|
+ // Verify allocation
|
|
|
+ assertTrue(alloc2.toString(),
|
|
|
+ check(alloc2, 14 * step, 16 * step, 100, 1024, 1));
|
|
|
+
|
|
|
+ agentLeft.deleteReservation(reservationID1, "u1", plan);
|
|
|
+ agentLeft.deleteReservation(reservationID2, "u2", plan);
|
|
|
+
|
|
|
+ // Now try to allocate the same jobs with agentRight. The second
|
|
|
+ // job should fail
|
|
|
+ // Add 1st reservation
|
|
|
+ ReservationId reservationID3 =
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ agentRight.createReservation(reservationID3, "u1", plan, rr1);
|
|
|
+
|
|
|
+ // CHECK: allocation was accepted
|
|
|
+ assertTrue("Agent-based allocation failed", reservationID3 != null);
|
|
|
+ assertTrue("Agent-based allocation failed", plan.getAllReservations()
|
|
|
+ .size() == numJobsInScenario + 1);
|
|
|
+
|
|
|
+ // Add 2nd reservation
|
|
|
+ try {
|
|
|
+ ReservationId reservationID4 =
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ agentRight.createReservation(reservationID4, "u2", plan, rr2);
|
|
|
+ fail();
|
|
|
+ } catch (PlanningException e) {
|
|
|
+ // Expected failure
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testValidateOrderNoGap() {
|
|
|
+
|
|
|
+ //
|
|
|
+ // Initialize allocations
|
|
|
+ //
|
|
|
+
|
|
|
+ RLESparseResourceAllocation allocation =
|
|
|
+ new RLESparseResourceAllocation(res);
|
|
|
+ allocation.addInterval(new ReservationInterval(10 * step, 13 * step),
|
|
|
+ Resource.newInstance(1024, 1));
|
|
|
+
|
|
|
+ // curAlloc
|
|
|
+ Map<ReservationInterval, Resource> curAlloc =
|
|
|
+ new HashMap<ReservationInterval, Resource>();
|
|
|
+
|
|
|
+ //
|
|
|
+ // Check cases
|
|
|
+ //
|
|
|
+
|
|
|
+ // 1. allocateLeft = false, succeed when there is no gap
|
|
|
+ curAlloc.clear();
|
|
|
+ curAlloc.put(new ReservationInterval(9 * step, 10 * step),
|
|
|
+ Resource.newInstance(1024, 1));
|
|
|
+ assertTrue("validateOrderNoFap() should have suceeded",
|
|
|
+ IterativePlanner.validateOrderNoGap(allocation, curAlloc, false));
|
|
|
+
|
|
|
+ // 2. allocateLeft = false, fail when curAlloc has a gap
|
|
|
+ curAlloc.put(new ReservationInterval(7 * step, 8 * step),
|
|
|
+ Resource.newInstance(1024, 1));
|
|
|
+ assertFalse("validateOrderNoGap() failed to identify a gap in curAlloc",
|
|
|
+ IterativePlanner.validateOrderNoGap(allocation, curAlloc, false));
|
|
|
+
|
|
|
+ // 3. allocateLeft = false, fail when there is a gap between curAlloc and
|
|
|
+ // allocations
|
|
|
+ curAlloc.clear();
|
|
|
+ curAlloc.put(new ReservationInterval(8 * step, 9 * step),
|
|
|
+ Resource.newInstance(1024, 1));
|
|
|
+ assertFalse("validateOrderNoGap() failed to identify a gap between "
|
|
|
+ + "allocations and curAlloc",
|
|
|
+ IterativePlanner.validateOrderNoGap(allocation, curAlloc, false));
|
|
|
+
|
|
|
+ // 4. allocateLeft = true, succeed when there is no gap
|
|
|
+ curAlloc.clear();
|
|
|
+ curAlloc.put(new ReservationInterval(13 * step, 14 * step),
|
|
|
+ Resource.newInstance(1024, 1));
|
|
|
+ assertTrue("validateOrderNoFap() should have suceeded",
|
|
|
+ IterativePlanner.validateOrderNoGap(allocation, curAlloc, true));
|
|
|
+
|
|
|
+ // 5. allocateLeft = true, fail when there is a gap between curAlloc and
|
|
|
+ // allocations
|
|
|
+ curAlloc.put(new ReservationInterval(15 * step, 16 * step),
|
|
|
+ Resource.newInstance(1024, 1));
|
|
|
+ assertFalse("validateOrderNoGap() failed to identify a gap in curAlloc",
|
|
|
+ IterativePlanner.validateOrderNoGap(allocation, curAlloc, true));
|
|
|
+
|
|
|
+ // 6. allocateLeft = true, fail when curAlloc has a gap
|
|
|
+ curAlloc.clear();
|
|
|
+ curAlloc.put(new ReservationInterval(14 * step, 15 * step),
|
|
|
+ Resource.newInstance(1024, 1));
|
|
|
+ assertFalse("validateOrderNoGap() failed to identify a gap between "
|
|
|
+ + "allocations and curAlloc",
|
|
|
+ IterativePlanner.validateOrderNoGap(allocation, curAlloc, true));
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testGetDurationInterval() throws PlanningException {
|
|
|
+
|
|
|
+ DurationInterval durationInterval = null;
|
|
|
+
|
|
|
+ // Create netRLERes:
|
|
|
+ // - 4GB & 4VC between [10,20) and [30,40)
|
|
|
+ // - 8GB & 8VC between [20,30)
|
|
|
+ RLESparseResourceAllocation netRLERes =
|
|
|
+ new RLESparseResourceAllocation(res);
|
|
|
+ netRLERes.addInterval(
|
|
|
+ new ReservationInterval(10 * step, 40 * step),
|
|
|
+ Resource.newInstance(4096, 4)
|
|
|
+ );
|
|
|
+ netRLERes.addInterval(
|
|
|
+ new ReservationInterval(20 * step, 30 * step),
|
|
|
+ Resource.newInstance(4096, 4)
|
|
|
+ );
|
|
|
+
|
|
|
+ // Create planLoads:
|
|
|
+ // - 5GB & 5VC between [20,30)
|
|
|
+ RLESparseResourceAllocation planLoads =
|
|
|
+ new RLESparseResourceAllocation(res);
|
|
|
+ planLoads.addInterval(
|
|
|
+ new ReservationInterval(20 * step, 30 * step),
|
|
|
+ Resource.newInstance(5120, 5)
|
|
|
+ );
|
|
|
+
|
|
|
+ // Create planModifications:
|
|
|
+ // - 1GB & 1VC between [25,35)
|
|
|
+ RLESparseResourceAllocation planModifications =
|
|
|
+ new RLESparseResourceAllocation(res);
|
|
|
+ planModifications.addInterval(
|
|
|
+ new ReservationInterval(25 * step, 35 * step),
|
|
|
+ Resource.newInstance(1024, 1)
|
|
|
+ );
|
|
|
+
|
|
|
+ // Set requested resources
|
|
|
+ Resource requestedResources = Resource.newInstance(1024, 1);
|
|
|
+
|
|
|
+
|
|
|
+ // 1.
|
|
|
+ // currLoad: should start at 20*step, end at 30*step with a null value
|
|
|
+ // (in getTotalCost(), after the for loop we will have loadPrev == null
|
|
|
+ // netAvailableResources: should start exactly at startTime (10*step),
|
|
|
+ // end exactly at endTime (30*step) with a null value
|
|
|
+ durationInterval =
|
|
|
+ StageAllocatorLowCostAligned.getDurationInterval(10*step, 30*step,
|
|
|
+ planLoads, planModifications, clusterCapacity, netRLERes, res, step,
|
|
|
+ requestedResources);
|
|
|
+ assertEquals(durationInterval.numCanFit(), 4);
|
|
|
+ assertEquals(durationInterval.getTotalCost(), 0.55, 0.00001);
|
|
|
+
|
|
|
+ // 2.
|
|
|
+ // currLoad: should start at 20*step, end at 31*step with a null value
|
|
|
+ // (in getTotalCost, after the for loop we will have loadPrev == null)
|
|
|
+ // netAvailableResources: should start exactly at startTime (10*step),
|
|
|
+ // end exactly at endTime (31*step) with a null value
|
|
|
+ durationInterval =
|
|
|
+ StageAllocatorLowCostAligned.getDurationInterval(10*step, 31*step,
|
|
|
+ planLoads, planModifications, clusterCapacity, netRLERes, res, step,
|
|
|
+ requestedResources);
|
|
|
+ System.out.println(durationInterval);
|
|
|
+ assertEquals(durationInterval.numCanFit(), 3);
|
|
|
+ assertEquals(durationInterval.getTotalCost(), 0.56, 0.00001);
|
|
|
+
|
|
|
+ // 3.
|
|
|
+ // currLoad: should start at 20*step, end at 30*step with a null value
|
|
|
+ // (in getTotalCost, after the for loop we will have loadPrev == null)
|
|
|
+ // netAvailableResources: should start exactly startTime (15*step),
|
|
|
+ // end exactly at endTime (30*step) with a null value
|
|
|
+ durationInterval =
|
|
|
+ StageAllocatorLowCostAligned.getDurationInterval(15*step, 30*step,
|
|
|
+ planLoads, planModifications, clusterCapacity, netRLERes, res, step,
|
|
|
+ requestedResources);
|
|
|
+ assertEquals(durationInterval.numCanFit(), 4);
|
|
|
+ assertEquals(durationInterval.getTotalCost(), 0.55, 0.00001);
|
|
|
+
|
|
|
+ // 4.
|
|
|
+ // currLoad: should start at 20*step, end at 31*step with a null value
|
|
|
+ // (in getTotalCost, after the for loop we will have loadPrev == null)
|
|
|
+ // netAvailableResources: should start exactly at startTime (15*step),
|
|
|
+ // end exactly at endTime (31*step) with a value other than null
|
|
|
+ durationInterval =
|
|
|
+ StageAllocatorLowCostAligned.getDurationInterval(15*step, 31*step,
|
|
|
+ planLoads, planModifications, clusterCapacity, netRLERes, res, step,
|
|
|
+ requestedResources);
|
|
|
+ System.out.println(durationInterval);
|
|
|
+ assertEquals(durationInterval.numCanFit(), 3);
|
|
|
+ assertEquals(durationInterval.getTotalCost(), 0.56, 0.00001);
|
|
|
+
|
|
|
+ // 5.
|
|
|
+ // currLoad: should only contain one entry at startTime
|
|
|
+ // (22*step), therefore loadPrev != null and we should enter the if
|
|
|
+ // condition after the for loop in getTotalCost
|
|
|
+ // netAvailableResources: should only contain one entry at startTime
|
|
|
+ // (22*step)
|
|
|
+ durationInterval =
|
|
|
+ StageAllocatorLowCostAligned.getDurationInterval(22*step, 23*step,
|
|
|
+ planLoads, planModifications, clusterCapacity, netRLERes, res, step,
|
|
|
+ requestedResources);
|
|
|
+ System.out.println(durationInterval);
|
|
|
+ assertEquals(durationInterval.numCanFit(), 8);
|
|
|
+ assertEquals(durationInterval.getTotalCost(), 0.05, 0.00001);
|
|
|
+
|
|
|
+ // 6.
|
|
|
+ // currLoad: should start at 39*step, end at 41*step with a null value
|
|
|
+ // (in getTotalCost, after the for loop we will have loadPrev == null)
|
|
|
+ // netAvailableResources: should start exactly at startTime (39*step),
|
|
|
+ // end exactly at endTime (41*step) with a null value
|
|
|
+ durationInterval =
|
|
|
+ StageAllocatorLowCostAligned.getDurationInterval(39*step, 41*step,
|
|
|
+ planLoads, planModifications, clusterCapacity, netRLERes, res, step,
|
|
|
+ requestedResources);
|
|
|
+ System.out.println(durationInterval);
|
|
|
+ assertEquals(durationInterval.numCanFit(), 0);
|
|
|
+ assertEquals(durationInterval.getTotalCost(), 0, 0.00001);
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
@Before
|
|
|
public void setup() throws Exception {
|
|
|
|
|
|
// Initialize random seed
|
|
|
long seed = rand.nextLong();
|
|
|
rand.setSeed(seed);
|
|
|
- Log.info("Running with seed: " + seed);
|
|
|
+ Log.getLog().info("Running with seed: " + seed);
|
|
|
|
|
|
// Set cluster parameters
|
|
|
long timeWindow = 1000000L;
|
|
@@ -709,16 +1038,15 @@ public class TestAlignedPlanner {
|
|
|
int capacityCores = 100;
|
|
|
step = 60000L;
|
|
|
|
|
|
- Resource clusterCapacity = Resource.newInstance(capacityMem, capacityCores);
|
|
|
+ clusterCapacity = Resource.newInstance(capacityMem, capacityCores);
|
|
|
|
|
|
String reservationQ =
|
|
|
ReservationSystemTestUtil.getFullReservationQueueName();
|
|
|
float instConstraint = 100;
|
|
|
float avgConstraint = 100;
|
|
|
|
|
|
- ReservationSchedulerConfiguration conf =
|
|
|
- ReservationSystemTestUtil.createConf(reservationQ, timeWindow,
|
|
|
- instConstraint, avgConstraint);
|
|
|
+ ReservationSchedulerConfiguration conf = ReservationSystemTestUtil
|
|
|
+ .createConf(reservationQ, timeWindow, instConstraint, avgConstraint);
|
|
|
|
|
|
CapacityOverTimePolicy policy = new CapacityOverTimePolicy();
|
|
|
policy.init(reservationQ, conf);
|
|
@@ -728,14 +1056,19 @@ public class TestAlignedPlanner {
|
|
|
|
|
|
conf.setInt(AlignedPlannerWithGreedy.SMOOTHNESS_FACTOR,
|
|
|
AlignedPlannerWithGreedy.DEFAULT_SMOOTHNESS_FACTOR);
|
|
|
+ conf.setBoolean(ReservationAgent.FAVOR_EARLY_ALLOCATION, false);
|
|
|
+
|
|
|
// Set planning agent
|
|
|
- agent = new AlignedPlannerWithGreedy();
|
|
|
- agent.init(conf);
|
|
|
+ agentRight = new AlignedPlannerWithGreedy();
|
|
|
+ agentRight.init(conf);
|
|
|
+
|
|
|
+ conf.setBoolean(ReservationAgent.FAVOR_EARLY_ALLOCATION, true);
|
|
|
+ agentLeft = new AlignedPlannerWithGreedy();
|
|
|
+ agentLeft.init(conf);
|
|
|
|
|
|
// Create Plan
|
|
|
- plan =
|
|
|
- new InMemoryPlan(queueMetrics, policy, agent, clusterCapacity, step,
|
|
|
- res, minAlloc, maxAlloc, "dedicated", null, true, context);
|
|
|
+ plan = new InMemoryPlan(queueMetrics, policy, agentRight, clusterCapacity,
|
|
|
+ step, res, minAlloc, maxAlloc, "dedicated", null, true, context);
|
|
|
}
|
|
|
|
|
|
private int initializeScenario1() throws PlanningException {
|