|
@@ -24,6 +24,7 @@ import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.Map;
|
|
|
+import java.util.Set;
|
|
|
|
|
|
import org.apache.hadoop.yarn.api.records.ReservationDefinition;
|
|
|
import org.apache.hadoop.yarn.api.records.ReservationId;
|
|
@@ -98,19 +99,11 @@ public class TestInMemoryPlan {
|
|
|
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
|
|
|
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
|
|
|
ReservationId reservationID =
|
|
|
- ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
int[] alloc = { 10, 10, 10, 10, 10, 10 };
|
|
|
int start = 100;
|
|
|
- Map<ReservationInterval, ReservationRequest> allocations =
|
|
|
- generateAllocation(start, alloc, false);
|
|
|
- ReservationDefinition rDef =
|
|
|
- createSimpleReservationDefinition(start, start + alloc.length,
|
|
|
- alloc.length, allocations.values());
|
|
|
- Map<ReservationInterval, Resource> allocs =
|
|
|
- ReservationSystemUtil.toResources(allocations);
|
|
|
- ReservationAllocation rAllocation =
|
|
|
- new InMemoryReservationAllocation(reservationID, rDef, user, planName,
|
|
|
- start, start + alloc.length, allocs, resCalc, minAlloc);
|
|
|
+ ReservationAllocation rAllocation = createReservationAllocation
|
|
|
+ (reservationID, start, alloc);
|
|
|
Assert.assertNull(plan.getReservationById(reservationID));
|
|
|
try {
|
|
|
plan.addReservation(rAllocation, false);
|
|
@@ -139,19 +132,11 @@ public class TestInMemoryPlan {
|
|
|
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
|
|
|
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
|
|
|
ReservationId reservationID =
|
|
|
- ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
int[] alloc = {};
|
|
|
int start = 100;
|
|
|
- Map<ReservationInterval, ReservationRequest> allocations =
|
|
|
- new HashMap<ReservationInterval, ReservationRequest>();
|
|
|
- ReservationDefinition rDef =
|
|
|
- createSimpleReservationDefinition(start, start + alloc.length,
|
|
|
- alloc.length, allocations.values());
|
|
|
- Map<ReservationInterval, Resource> allocs = ReservationSystemUtil.toResources
|
|
|
- (allocations);
|
|
|
- ReservationAllocation rAllocation =
|
|
|
- new InMemoryReservationAllocation(reservationID, rDef, user, planName,
|
|
|
- start, start + alloc.length, allocs, resCalc, minAlloc);
|
|
|
+ ReservationAllocation rAllocation = createReservationAllocation
|
|
|
+ (reservationID, start, alloc);
|
|
|
Assert.assertNull(plan.getReservationById(reservationID));
|
|
|
try {
|
|
|
plan.addReservation(rAllocation, false);
|
|
@@ -167,19 +152,11 @@ public class TestInMemoryPlan {
|
|
|
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
|
|
|
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
|
|
|
ReservationId reservationID =
|
|
|
- ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
int[] alloc = { 10, 10, 10, 10, 10, 10 };
|
|
|
int start = 100;
|
|
|
- Map<ReservationInterval, ReservationRequest> allocations =
|
|
|
- generateAllocation(start, alloc, false);
|
|
|
- ReservationDefinition rDef =
|
|
|
- createSimpleReservationDefinition(start, start + alloc.length,
|
|
|
- alloc.length, allocations.values());
|
|
|
- Map<ReservationInterval, Resource> allocs = ReservationSystemUtil.toResources
|
|
|
- (allocations);
|
|
|
- ReservationAllocation rAllocation =
|
|
|
- new InMemoryReservationAllocation(reservationID, rDef, user, planName,
|
|
|
- start, start + alloc.length, allocs, resCalc, minAlloc);
|
|
|
+ ReservationAllocation rAllocation = createReservationAllocation
|
|
|
+ (reservationID, start, alloc);
|
|
|
Assert.assertNull(plan.getReservationById(reservationID));
|
|
|
try {
|
|
|
plan.addReservation(rAllocation, false);
|
|
@@ -211,16 +188,8 @@ public class TestInMemoryPlan {
|
|
|
// First add a reservation
|
|
|
int[] alloc = { 10, 10, 10, 10, 10, 10 };
|
|
|
int start = 100;
|
|
|
- Map<ReservationInterval, ReservationRequest> allocations =
|
|
|
- generateAllocation(start, alloc, false);
|
|
|
- ReservationDefinition rDef =
|
|
|
- createSimpleReservationDefinition(start, start + alloc.length,
|
|
|
- alloc.length, allocations.values());
|
|
|
- Map<ReservationInterval, Resource> allocs = ReservationSystemUtil.toResources
|
|
|
- (allocations);
|
|
|
- ReservationAllocation rAllocation =
|
|
|
- new InMemoryReservationAllocation(reservationID, rDef, user, planName,
|
|
|
- start, start + alloc.length, allocs, resCalc, minAlloc);
|
|
|
+ ReservationAllocation rAllocation = createReservationAllocation
|
|
|
+ (reservationID, start, alloc);
|
|
|
Assert.assertNull(plan.getReservationById(reservationID));
|
|
|
try {
|
|
|
plan.addReservation(rAllocation, false);
|
|
@@ -241,16 +210,8 @@ public class TestInMemoryPlan {
|
|
|
// Now update it
|
|
|
start = 110;
|
|
|
int[] updatedAlloc = { 0, 5, 10, 10, 5, 0 };
|
|
|
- allocations = generateAllocation(start, updatedAlloc, true);
|
|
|
- rDef =
|
|
|
- createSimpleReservationDefinition(start, start + updatedAlloc.length,
|
|
|
- updatedAlloc.length, allocations.values());
|
|
|
- Map<ReservationInterval, Resource> updatedAllocs =
|
|
|
- ReservationSystemUtil.toResources(allocations);
|
|
|
- rAllocation =
|
|
|
- new InMemoryReservationAllocation(reservationID, rDef, user, planName,
|
|
|
- start, start + updatedAlloc.length, updatedAllocs, resCalc,
|
|
|
- minAlloc);
|
|
|
+ rAllocation = createReservationAllocation(reservationID, start,
|
|
|
+ updatedAlloc, true);
|
|
|
try {
|
|
|
plan.updateReservation(rAllocation);
|
|
|
} catch (PlanningException e) {
|
|
@@ -282,16 +243,8 @@ public class TestInMemoryPlan {
|
|
|
// Try to update a reservation without adding
|
|
|
int[] alloc = { 10, 10, 10, 10, 10, 10 };
|
|
|
int start = 100;
|
|
|
- Map<ReservationInterval, ReservationRequest> allocations =
|
|
|
- generateAllocation(start, alloc, false);
|
|
|
- ReservationDefinition rDef =
|
|
|
- createSimpleReservationDefinition(start, start + alloc.length,
|
|
|
- alloc.length, allocations.values());
|
|
|
- Map<ReservationInterval, Resource> allocs =
|
|
|
- ReservationSystemUtil.toResources(allocations);
|
|
|
ReservationAllocation rAllocation =
|
|
|
- new InMemoryReservationAllocation(reservationID, rDef, user, planName,
|
|
|
- start, start + alloc.length, allocs, resCalc, minAlloc);
|
|
|
+ createReservationAllocation(reservationID, start, alloc);
|
|
|
Assert.assertNull(plan.getReservationById(reservationID));
|
|
|
try {
|
|
|
plan.updateReservation(rAllocation);
|
|
@@ -314,16 +267,8 @@ public class TestInMemoryPlan {
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
int[] alloc = { 10, 10, 10, 10, 10, 10 };
|
|
|
int start = 100;
|
|
|
- Map<ReservationInterval, ReservationRequest> allocations =
|
|
|
- generateAllocation(start, alloc, true);
|
|
|
- ReservationDefinition rDef =
|
|
|
- createSimpleReservationDefinition(start, start + alloc.length,
|
|
|
- alloc.length, allocations.values());
|
|
|
- Map<ReservationInterval, Resource> allocs =
|
|
|
- ReservationSystemUtil.toResources(allocations);
|
|
|
ReservationAllocation rAllocation =
|
|
|
- new InMemoryReservationAllocation(reservationID, rDef, user, planName,
|
|
|
- start, start + alloc.length, allocs, resCalc, minAlloc);
|
|
|
+ createReservationAllocation(reservationID, start, alloc, true);
|
|
|
Assert.assertNull(plan.getReservationById(reservationID));
|
|
|
try {
|
|
|
plan.addReservation(rAllocation, false);
|
|
@@ -391,17 +336,8 @@ public class TestInMemoryPlan {
|
|
|
// First add a reservation
|
|
|
int[] alloc1 = { 10, 10, 10, 10, 10, 10 };
|
|
|
int start = 100;
|
|
|
- Map<ReservationInterval, ReservationRequest> allocations1 =
|
|
|
- generateAllocation(start, alloc1, false);
|
|
|
- ReservationDefinition rDef1 =
|
|
|
- createSimpleReservationDefinition(start, start + alloc1.length,
|
|
|
- alloc1.length, allocations1.values());
|
|
|
- Map<ReservationInterval, Resource> allocs1 =
|
|
|
- ReservationSystemUtil.toResources(allocations1);
|
|
|
ReservationAllocation rAllocation =
|
|
|
- new InMemoryReservationAllocation(reservationID1, rDef1, user,
|
|
|
- planName, start, start + alloc1.length, allocs1, resCalc,
|
|
|
- minAlloc);
|
|
|
+ createReservationAllocation(reservationID1, start, alloc1);
|
|
|
Assert.assertNull(plan.getReservationById(reservationID1));
|
|
|
try {
|
|
|
plan.addReservation(rAllocation, false);
|
|
@@ -416,17 +352,8 @@ public class TestInMemoryPlan {
|
|
|
ReservationId reservationID2 =
|
|
|
ReservationSystemTestUtil.getNewReservationId();
|
|
|
int[] alloc2 = { 0, 5, 10, 5, 0 };
|
|
|
- Map<ReservationInterval, ReservationRequest> allocations2 =
|
|
|
- generateAllocation(start, alloc2, true);
|
|
|
- ReservationDefinition rDef2 =
|
|
|
- createSimpleReservationDefinition(start, start + alloc2.length,
|
|
|
- alloc2.length, allocations2.values());
|
|
|
- Map<ReservationInterval, Resource> allocs2 =
|
|
|
- ReservationSystemUtil.toResources(allocations2);
|
|
|
rAllocation =
|
|
|
- new InMemoryReservationAllocation(reservationID2, rDef2, user,
|
|
|
- planName, start, start + alloc2.length, allocs2, resCalc,
|
|
|
- minAlloc);
|
|
|
+ createReservationAllocation(reservationID2, start, alloc2, true);
|
|
|
Assert.assertNull(plan.getReservationById(reservationID2));
|
|
|
try {
|
|
|
plan.addReservation(rAllocation, false);
|
|
@@ -482,6 +409,192 @@ public class TestInMemoryPlan {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testGetReservationsById() {
|
|
|
+ Plan plan =
|
|
|
+ new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
|
|
|
+ resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
|
|
|
+ ReservationId reservationID =
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ int[] alloc = {10, 10, 10, 10, 10, 10};
|
|
|
+ int start = 100;
|
|
|
+ ReservationAllocation rAllocation = createReservationAllocation
|
|
|
+ (reservationID, start, alloc);
|
|
|
+ Assert.assertNull(plan.getReservationById(reservationID));
|
|
|
+ try {
|
|
|
+ plan.addReservation(rAllocation, false);
|
|
|
+ } catch (PlanningException e) {
|
|
|
+ Assert.fail(e.getMessage());
|
|
|
+ }
|
|
|
+
|
|
|
+ // Verify that get by reservation id works.
|
|
|
+ Set<ReservationAllocation> rAllocations =
|
|
|
+ plan.getReservations(reservationID, null, "");
|
|
|
+ Assert.assertTrue(rAllocations.size() == 1);
|
|
|
+ Assert.assertTrue(rAllocation.compareTo(
|
|
|
+ (ReservationAllocation) rAllocations.toArray()[0]) == 0);
|
|
|
+
|
|
|
+ // Verify that get by reservation id works even when time range
|
|
|
+ // and user is invalid.
|
|
|
+ ReservationInterval interval = new ReservationInterval(0, 0);
|
|
|
+ rAllocations = plan.getReservations(reservationID, interval, "invalid");
|
|
|
+ Assert.assertTrue(rAllocations.size() == 1);
|
|
|
+ Assert.assertTrue(rAllocation.compareTo(
|
|
|
+ (ReservationAllocation) rAllocations.toArray()[0]) == 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testGetReservationsByInvalidId() {
|
|
|
+ Plan plan =
|
|
|
+ new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
|
|
|
+ resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
|
|
|
+ ReservationId reservationID =
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ int[] alloc = {10, 10, 10, 10, 10, 10};
|
|
|
+ int start = 100;
|
|
|
+ ReservationAllocation rAllocation = createReservationAllocation
|
|
|
+ (reservationID, start, alloc);
|
|
|
+ Assert.assertNull(plan.getReservationById(reservationID));
|
|
|
+ try {
|
|
|
+ plan.addReservation(rAllocation, false);
|
|
|
+ } catch (PlanningException e) {
|
|
|
+ Assert.fail(e.getMessage());
|
|
|
+ }
|
|
|
+
|
|
|
+ // If reservationId is null, then nothing is returned.
|
|
|
+ ReservationId invalidReservationID =
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ Set<ReservationAllocation> rAllocations =
|
|
|
+ plan.getReservations(invalidReservationID, null, "");
|
|
|
+ Assert.assertTrue(rAllocations.size() == 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testGetReservationsByTimeInterval() {
|
|
|
+ Plan plan =
|
|
|
+ new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
|
|
|
+ resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
|
|
|
+ ReservationId reservationID =
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ int[] alloc = {10, 10, 10, 10, 10, 10};
|
|
|
+ int start = 100;
|
|
|
+ ReservationAllocation rAllocation = createReservationAllocation
|
|
|
+ (reservationID, start, alloc);
|
|
|
+ Assert.assertNull(plan.getReservationById(reservationID));
|
|
|
+ try {
|
|
|
+ plan.addReservation(rAllocation, false);
|
|
|
+ } catch (PlanningException e) {
|
|
|
+ Assert.fail(e.getMessage());
|
|
|
+ }
|
|
|
+
|
|
|
+ // Verify that get by time interval works if the selection interval
|
|
|
+ // completely overlaps with the allocation.
|
|
|
+ ReservationInterval interval = new ReservationInterval(rAllocation
|
|
|
+ .getStartTime(), rAllocation.getEndTime());
|
|
|
+ Set<ReservationAllocation> rAllocations =
|
|
|
+ plan.getReservations(null, interval, "");
|
|
|
+ Assert.assertTrue(rAllocations.size() == 1);
|
|
|
+ Assert.assertTrue(rAllocation.compareTo(
|
|
|
+ (ReservationAllocation) rAllocations.toArray()[0]) == 0);
|
|
|
+
|
|
|
+ // Verify that get by time interval works if the selection interval
|
|
|
+ // falls within the allocation
|
|
|
+ long duration = rAllocation.getEndTime() - rAllocation.getStartTime();
|
|
|
+ interval = new ReservationInterval(rAllocation.getStartTime() + duration
|
|
|
+ * (long)0.3, rAllocation.getEndTime() - duration * (long)0.3);
|
|
|
+ rAllocations = plan.getReservations(null, interval, "");
|
|
|
+ Assert.assertTrue(rAllocations.size() == 1);
|
|
|
+ Assert.assertTrue(rAllocation.compareTo(
|
|
|
+ (ReservationAllocation) rAllocations.toArray()[0]) == 0);
|
|
|
+
|
|
|
+ // Verify that get by time interval selects 1 allocation if the end
|
|
|
+ // time of the selection interval falls right at the start of the
|
|
|
+ // allocation.
|
|
|
+ interval = new ReservationInterval(0, rAllocation.getStartTime());
|
|
|
+ rAllocations = plan.getReservations(null, interval, "");
|
|
|
+ Assert.assertTrue(rAllocations.size() == 1);
|
|
|
+ Assert.assertTrue(rAllocation.compareTo(
|
|
|
+ (ReservationAllocation) rAllocations.toArray()[0]) == 0);
|
|
|
+
|
|
|
+ // Verify that get by time interval selects no reservations if the start
|
|
|
+ // time of the selection interval falls right at the end of the allocation.
|
|
|
+ interval = new ReservationInterval(rAllocation
|
|
|
+ .getEndTime(), Long.MAX_VALUE);
|
|
|
+ rAllocations = plan.getReservations(null, interval, "");
|
|
|
+ Assert.assertTrue(rAllocations.size() == 0);
|
|
|
+
|
|
|
+ // Verify that get by time interval selects no reservations if the
|
|
|
+ // selection interval and allocation interval do not overlap.
|
|
|
+ interval = new ReservationInterval(0, rAllocation.getStartTime() / 2);
|
|
|
+ rAllocations = plan.getReservations(null, interval, "");
|
|
|
+ Assert.assertTrue(rAllocations.size() == 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testGetReservationsAtTime() {
|
|
|
+ Plan plan =
|
|
|
+ new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
|
|
|
+ resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
|
|
|
+ ReservationId reservationID =
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ int[] alloc = {10, 10, 10, 10, 10, 10};
|
|
|
+ int start = 100;
|
|
|
+ ReservationAllocation rAllocation = createReservationAllocation
|
|
|
+ (reservationID, start, alloc);
|
|
|
+ Assert.assertNull(plan.getReservationById(reservationID));
|
|
|
+ try {
|
|
|
+ plan.addReservation(rAllocation, false);
|
|
|
+ } catch (PlanningException e) {
|
|
|
+ Assert.fail(e.getMessage());
|
|
|
+ }
|
|
|
+
|
|
|
+ Set<ReservationAllocation> rAllocations =
|
|
|
+ plan.getReservationsAtTime(rAllocation.getStartTime());
|
|
|
+ Assert.assertTrue(rAllocations.size() == 1);
|
|
|
+ Assert.assertTrue(rAllocation.compareTo(
|
|
|
+ (ReservationAllocation) rAllocations.toArray()[0]) == 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testGetReservationsWithNoInput() {
|
|
|
+ Plan plan =
|
|
|
+ new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
|
|
|
+ resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
|
|
|
+ ReservationId reservationID =
|
|
|
+ ReservationSystemTestUtil.getNewReservationId();
|
|
|
+ int[] alloc = {10, 10, 10, 10, 10, 10};
|
|
|
+ int start = 100;
|
|
|
+ ReservationAllocation rAllocation = createReservationAllocation
|
|
|
+ (reservationID, start, alloc);
|
|
|
+ Assert.assertNull(plan.getReservationById(reservationID));
|
|
|
+ try {
|
|
|
+ plan.addReservation(rAllocation, false);
|
|
|
+ } catch (PlanningException e) {
|
|
|
+ Assert.fail(e.getMessage());
|
|
|
+ }
|
|
|
+
|
|
|
+ // Verify that getReservations defaults to getting all reservations if no
|
|
|
+ // reservationID, time interval, and user is provided,
|
|
|
+ Set<ReservationAllocation> rAllocations =
|
|
|
+ plan.getReservations(null, null, "");
|
|
|
+ Assert.assertTrue(rAllocations.size() == 1);
|
|
|
+ Assert.assertTrue(rAllocation.compareTo(
|
|
|
+ (ReservationAllocation) rAllocations.toArray()[0]) == 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testGetReservationsWithNoReservation() {
|
|
|
+ Plan plan =
|
|
|
+ new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
|
|
|
+ resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
|
|
|
+ // Verify that get reservation returns no entries if no queries are made.
|
|
|
+
|
|
|
+ ReservationInterval interval = new ReservationInterval(0, Long.MAX_VALUE);
|
|
|
+ Set<ReservationAllocation> rAllocations =
|
|
|
+ plan.getReservations(null, interval, "");
|
|
|
+ Assert.assertTrue(rAllocations.size() == 0);
|
|
|
+ }
|
|
|
+
|
|
|
private void doAssertions(Plan plan, ReservationAllocation rAllocation) {
|
|
|
ReservationId reservationID = rAllocation.getReservationId();
|
|
|
Assert.assertNotNull(plan.getReservationById(reservationID));
|
|
@@ -528,4 +641,23 @@ public class TestInMemoryPlan {
|
|
|
return req;
|
|
|
}
|
|
|
|
|
|
+ private ReservationAllocation createReservationAllocation(ReservationId
|
|
|
+ reservationID, int start, int[] alloc) {
|
|
|
+ return createReservationAllocation(reservationID, start, alloc, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ private ReservationAllocation createReservationAllocation(ReservationId
|
|
|
+ reservationID, int start, int[] alloc, boolean isStep) {
|
|
|
+ Map<ReservationInterval, ReservationRequest> allocations =
|
|
|
+ generateAllocation(start, alloc, isStep);
|
|
|
+ ReservationDefinition rDef =
|
|
|
+ createSimpleReservationDefinition(start, start + alloc.length,
|
|
|
+ alloc.length, allocations.values());
|
|
|
+ Map<ReservationInterval, Resource> allocs =
|
|
|
+ ReservationSystemUtil.toResources(allocations);
|
|
|
+ return new InMemoryReservationAllocation(reservationID, rDef, user,
|
|
|
+ planName,
|
|
|
+ start, start + alloc.length, allocs, resCalc, minAlloc);
|
|
|
+ }
|
|
|
+
|
|
|
}
|