YARN-3800. Reduce storage footprint for ReservationAllocation. Contributed by Anubhav Dhoot.

This commit is contained in:
carlo curino 2015-07-09 16:47:35 -07:00
parent f4ca530c1c
commit 0e602fa3a1
14 changed files with 176 additions and 123 deletions

View File

@ -323,6 +323,9 @@ Release 2.8.0 - UNRELEASED
YARN-3827. Migrate YARN native build to new CMake framework (Alan Burlison YARN-3827. Migrate YARN native build to new CMake framework (Alan Burlison
via Colin P. McCabe) via Colin P. McCabe)
YARN-3800. Reduce storage footprint for ReservationAllocation. (Anubhav Dhoot
via curino)
OPTIMIZATIONS OPTIMIZATIONS
YARN-3339. TestDockerContainerExecutor should pull a single image and not YARN-3339. TestDockerContainerExecutor should pull a single image and not

View File

@ -97,8 +97,8 @@ private boolean computeAllocation(ReservationId reservationId, String user,
long curDeadline = deadline; long curDeadline = deadline;
long oldDeadline = -1; long oldDeadline = -1;
Map<ReservationInterval, ReservationRequest> allocations = Map<ReservationInterval, Resource> allocations =
new HashMap<ReservationInterval, ReservationRequest>(); new HashMap<ReservationInterval, Resource>();
RLESparseResourceAllocation tempAssigned = RLESparseResourceAllocation tempAssigned =
new RLESparseResourceAllocation(plan.getResourceCalculator(), new RLESparseResourceAllocation(plan.getResourceCalculator(),
plan.getMinimumAllocation()); plan.getMinimumAllocation());
@ -108,6 +108,8 @@ private boolean computeAllocation(ReservationId reservationId, String user,
ReservationRequestInterpreter type = contract.getReservationRequests() ReservationRequestInterpreter type = contract.getReservationRequests()
.getInterpreter(); .getInterpreter();
boolean hasGang = false;
// Iterate the stages in backward from deadline // Iterate the stages in backward from deadline
for (ListIterator<ReservationRequest> li = for (ListIterator<ReservationRequest> li =
stages.listIterator(stages.size()); li.hasPrevious();) { stages.listIterator(stages.size()); li.hasPrevious();) {
@ -117,8 +119,10 @@ private boolean computeAllocation(ReservationId reservationId, String user,
// validate the RR respect basic constraints // validate the RR respect basic constraints
validateInput(plan, currentReservationStage, totalCapacity); validateInput(plan, currentReservationStage, totalCapacity);
hasGang |= currentReservationStage.getConcurrency() > 1;
// run allocation for a single stage // run allocation for a single stage
Map<ReservationInterval, ReservationRequest> curAlloc = Map<ReservationInterval, Resource> curAlloc =
placeSingleStage(plan, tempAssigned, currentReservationStage, placeSingleStage(plan, tempAssigned, currentReservationStage,
earliestStart, curDeadline, oldReservation, totalCapacity); earliestStart, curDeadline, oldReservation, totalCapacity);
@ -178,8 +182,7 @@ private boolean computeAllocation(ReservationId reservationId, String user,
// create reservation with above allocations if not null/empty // create reservation with above allocations if not null/empty
ReservationRequest ZERO_RES = Resource ZERO_RES = Resource.newInstance(0, 0);
ReservationRequest.newInstance(Resource.newInstance(0, 0), 0);
long firstStartTime = findEarliestTime(allocations.keySet()); long firstStartTime = findEarliestTime(allocations.keySet());
@ -200,7 +203,7 @@ private boolean computeAllocation(ReservationId reservationId, String user,
new InMemoryReservationAllocation(reservationId, contract, user, new InMemoryReservationAllocation(reservationId, contract, user,
plan.getQueueName(), firstStartTime, plan.getQueueName(), firstStartTime,
findLatestTime(allocations.keySet()), allocations, findLatestTime(allocations.keySet()), allocations,
plan.getResourceCalculator(), plan.getMinimumAllocation()); plan.getResourceCalculator(), plan.getMinimumAllocation(), hasGang);
if (oldReservation != null) { if (oldReservation != null) {
return plan.updateReservation(capReservation); return plan.updateReservation(capReservation);
} else { } else {
@ -242,13 +245,13 @@ private void validateInput(Plan plan, ReservationRequest rr,
* previous instant in time until the time-window is exhausted or we placed * previous instant in time until the time-window is exhausted or we placed
* all the user request. * all the user request.
*/ */
private Map<ReservationInterval, ReservationRequest> placeSingleStage( private Map<ReservationInterval, Resource> placeSingleStage(
Plan plan, RLESparseResourceAllocation tempAssigned, Plan plan, RLESparseResourceAllocation tempAssigned,
ReservationRequest rr, long earliestStart, long curDeadline, ReservationRequest rr, long earliestStart, long curDeadline,
ReservationAllocation oldResAllocation, final Resource totalCapacity) { ReservationAllocation oldResAllocation, final Resource totalCapacity) {
Map<ReservationInterval, ReservationRequest> allocationRequests = Map<ReservationInterval, Resource> allocationRequests =
new HashMap<ReservationInterval, ReservationRequest>(); new HashMap<ReservationInterval, Resource>();
// compute the gang as a resource and get the duration // compute the gang as a resource and get the duration
Resource gang = Resources.multiply(rr.getCapability(), rr.getConcurrency()); Resource gang = Resources.multiply(rr.getCapability(), rr.getConcurrency());
@ -322,7 +325,7 @@ private Map<ReservationInterval, ReservationRequest> placeSingleStage(
ReservationInterval reservationInt = ReservationInterval reservationInt =
new ReservationInterval(curDeadline - dur, curDeadline); new ReservationInterval(curDeadline - dur, curDeadline);
ReservationRequest reservationRes = ReservationRequest reservationRequest =
ReservationRequest.newInstance(rr.getCapability(), ReservationRequest.newInstance(rr.getCapability(),
rr.getConcurrency() * maxGang, rr.getConcurrency(), rr.getConcurrency() * maxGang, rr.getConcurrency(),
rr.getDuration()); rr.getDuration());
@ -331,6 +334,8 @@ private Map<ReservationInterval, ReservationRequest> placeSingleStage(
// placing other ReservationRequest within the same // placing other ReservationRequest within the same
// ReservationDefinition, // ReservationDefinition,
// and we must avoid double-counting the available resources // and we must avoid double-counting the available resources
final Resource reservationRes = ReservationSystemUtil.toResource(
reservationRequest);
tempAssigned.addInterval(reservationInt, reservationRes); tempAssigned.addInterval(reservationInt, reservationRes);
allocationRequests.put(reservationInt, reservationRes); allocationRequests.put(reservationInt, reservationRes);
@ -350,7 +355,7 @@ private Map<ReservationInterval, ReservationRequest> placeSingleStage(
// If we are here is becasue we did not manage to satisfy this request. // If we are here is becasue we did not manage to satisfy this request.
// So we need to remove unwanted side-effect from tempAssigned (needed // So we need to remove unwanted side-effect from tempAssigned (needed
// for ANY). // for ANY).
for (Map.Entry<ReservationInterval, ReservationRequest> tempAllocation : for (Map.Entry<ReservationInterval, Resource> tempAllocation :
allocationRequests.entrySet()) { allocationRequests.entrySet()) {
tempAssigned.removeInterval(tempAllocation.getKey(), tempAssigned.removeInterval(tempAllocation.getKey(),
tempAllocation.getValue()); tempAllocation.getValue());

View File

@ -31,7 +31,6 @@
import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.yarn.api.records.ReservationId; import org.apache.hadoop.yarn.api.records.ReservationId;
import org.apache.hadoop.yarn.api.records.ReservationRequest;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException; import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
@ -110,7 +109,7 @@ public QueueMetrics getQueueMetrics() {
private void incrementAllocation(ReservationAllocation reservation) { private void incrementAllocation(ReservationAllocation reservation) {
assert (readWriteLock.isWriteLockedByCurrentThread()); assert (readWriteLock.isWriteLockedByCurrentThread());
Map<ReservationInterval, ReservationRequest> allocationRequests = Map<ReservationInterval, Resource> allocationRequests =
reservation.getAllocationRequests(); reservation.getAllocationRequests();
// check if we have encountered the user earlier and if not add an entry // check if we have encountered the user earlier and if not add an entry
String user = reservation.getUser(); String user = reservation.getUser();
@ -119,7 +118,7 @@ private void incrementAllocation(ReservationAllocation reservation) {
resAlloc = new RLESparseResourceAllocation(resCalc, minAlloc); resAlloc = new RLESparseResourceAllocation(resCalc, minAlloc);
userResourceAlloc.put(user, resAlloc); userResourceAlloc.put(user, resAlloc);
} }
for (Map.Entry<ReservationInterval, ReservationRequest> r : allocationRequests for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
.entrySet()) { .entrySet()) {
resAlloc.addInterval(r.getKey(), r.getValue()); resAlloc.addInterval(r.getKey(), r.getValue());
rleSparseVector.addInterval(r.getKey(), r.getValue()); rleSparseVector.addInterval(r.getKey(), r.getValue());
@ -128,11 +127,11 @@ private void incrementAllocation(ReservationAllocation reservation) {
private void decrementAllocation(ReservationAllocation reservation) { private void decrementAllocation(ReservationAllocation reservation) {
assert (readWriteLock.isWriteLockedByCurrentThread()); assert (readWriteLock.isWriteLockedByCurrentThread());
Map<ReservationInterval, ReservationRequest> allocationRequests = Map<ReservationInterval, Resource> allocationRequests =
reservation.getAllocationRequests(); reservation.getAllocationRequests();
String user = reservation.getUser(); String user = reservation.getUser();
RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user); RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
for (Map.Entry<ReservationInterval, ReservationRequest> r : allocationRequests for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
.entrySet()) { .entrySet()) {
resAlloc.removeInterval(r.getKey(), r.getValue()); resAlloc.removeInterval(r.getKey(), r.getValue());
rleSparseVector.removeInterval(r.getKey(), r.getValue()); rleSparseVector.removeInterval(r.getKey(), r.getValue());

View File

@ -22,7 +22,6 @@
import org.apache.hadoop.yarn.api.records.ReservationDefinition; import org.apache.hadoop.yarn.api.records.ReservationDefinition;
import org.apache.hadoop.yarn.api.records.ReservationId; import org.apache.hadoop.yarn.api.records.ReservationId;
import org.apache.hadoop.yarn.api.records.ReservationRequest;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
@ -40,7 +39,7 @@ class InMemoryReservationAllocation implements ReservationAllocation {
private final ReservationDefinition contract; private final ReservationDefinition contract;
private final long startTime; private final long startTime;
private final long endTime; private final long endTime;
private final Map<ReservationInterval, ReservationRequest> allocationRequests; private final Map<ReservationInterval, Resource> allocationRequests;
private boolean hasGang = false; private boolean hasGang = false;
private long acceptedAt = -1; private long acceptedAt = -1;
@ -49,22 +48,29 @@ class InMemoryReservationAllocation implements ReservationAllocation {
InMemoryReservationAllocation(ReservationId reservationID, InMemoryReservationAllocation(ReservationId reservationID,
ReservationDefinition contract, String user, String planName, ReservationDefinition contract, String user, String planName,
long startTime, long endTime, long startTime, long endTime,
Map<ReservationInterval, ReservationRequest> allocationRequests, Map<ReservationInterval, Resource> allocations,
ResourceCalculator calculator, Resource minAlloc) { ResourceCalculator calculator, Resource minAlloc) {
this(reservationID, contract, user, planName, startTime, endTime,
allocations, calculator, minAlloc, false);
}
InMemoryReservationAllocation(ReservationId reservationID,
ReservationDefinition contract, String user, String planName,
long startTime, long endTime,
Map<ReservationInterval, Resource> allocations,
ResourceCalculator calculator, Resource minAlloc, boolean hasGang) {
this.contract = contract; this.contract = contract;
this.startTime = startTime; this.startTime = startTime;
this.endTime = endTime; this.endTime = endTime;
this.reservationID = reservationID; this.reservationID = reservationID;
this.user = user; this.user = user;
this.allocationRequests = allocationRequests; this.allocationRequests = allocations;
this.planName = planName; this.planName = planName;
this.hasGang = hasGang;
resourcesOverTime = new RLESparseResourceAllocation(calculator, minAlloc); resourcesOverTime = new RLESparseResourceAllocation(calculator, minAlloc);
for (Map.Entry<ReservationInterval, ReservationRequest> r : allocationRequests for (Map.Entry<ReservationInterval, Resource> r : allocations
.entrySet()) { .entrySet()) {
resourcesOverTime.addInterval(r.getKey(), r.getValue()); resourcesOverTime.addInterval(r.getKey(), r.getValue());
if (r.getValue().getConcurrency() > 1) {
hasGang = true;
}
} }
} }
@ -89,7 +95,7 @@ public long getEndTime() {
} }
@Override @Override
public Map<ReservationInterval, ReservationRequest> getAllocationRequests() { public Map<ReservationInterval, Resource> getAllocationRequests() {
return Collections.unmodifiableMap(allocationRequests); return Collections.unmodifiableMap(allocationRequests);
} }

View File

@ -21,7 +21,6 @@
import java.io.IOException; import java.io.IOException;
import java.io.StringWriter; import java.io.StringWriter;
import java.util.Iterator; import java.util.Iterator;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.NavigableMap; import java.util.NavigableMap;
@ -31,9 +30,7 @@
import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.yarn.api.records.ReservationRequest;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
@ -80,14 +77,11 @@ private boolean isSameAsNext(Long key, Resource capacity) {
* *
* @param reservationInterval the interval for which the resource is to be * @param reservationInterval the interval for which the resource is to be
* added * added
* @param capacity the resource to be added * @param totCap the resource to be added
* @return true if addition is successful, false otherwise * @return true if addition is successful, false otherwise
*/ */
public boolean addInterval(ReservationInterval reservationInterval, public boolean addInterval(ReservationInterval reservationInterval,
ReservationRequest capacity) { Resource totCap) {
Resource totCap =
Resources.multiply(capacity.getCapability(),
(float) capacity.getNumContainers());
if (totCap.equals(ZERO_RESOURCE)) { if (totCap.equals(ZERO_RESOURCE)) {
return true; return true;
} }
@ -142,45 +136,16 @@ public boolean addInterval(ReservationInterval reservationInterval,
} }
} }
/**
* Add multiple resources for the specified interval
*
* @param reservationInterval the interval for which the resource is to be
* added
* @param ReservationRequests the resources to be added
* @param clusterResource the total resources in the cluster
* @return true if addition is successful, false otherwise
*/
public boolean addCompositeInterval(ReservationInterval reservationInterval,
List<ReservationRequest> ReservationRequests, Resource clusterResource) {
ReservationRequest aggregateReservationRequest =
Records.newRecord(ReservationRequest.class);
Resource capacity = Resource.newInstance(0, 0);
for (ReservationRequest ReservationRequest : ReservationRequests) {
Resources.addTo(capacity, Resources.multiply(
ReservationRequest.getCapability(),
ReservationRequest.getNumContainers()));
}
aggregateReservationRequest.setNumContainers((int) Math.ceil(Resources
.divide(resourceCalculator, clusterResource, capacity, minAlloc)));
aggregateReservationRequest.setCapability(minAlloc);
return addInterval(reservationInterval, aggregateReservationRequest);
}
/** /**
* Removes a resource for the specified interval * Removes a resource for the specified interval
* *
* @param reservationInterval the interval for which the resource is to be * @param reservationInterval the interval for which the resource is to be
* removed * removed
* @param capacity the resource to be removed * @param totCap the resource to be removed
* @return true if removal is successful, false otherwise * @return true if removal is successful, false otherwise
*/ */
public boolean removeInterval(ReservationInterval reservationInterval, public boolean removeInterval(ReservationInterval reservationInterval,
ReservationRequest capacity) { Resource totCap) {
Resource totCap =
Resources.multiply(capacity.getCapability(),
(float) capacity.getNumContainers());
if (totCap.equals(ZERO_RESOURCE)) { if (totCap.equals(ZERO_RESOURCE)) {
return true; return true;
} }

View File

@ -22,7 +22,6 @@
import org.apache.hadoop.yarn.api.records.ReservationDefinition; import org.apache.hadoop.yarn.api.records.ReservationDefinition;
import org.apache.hadoop.yarn.api.records.ReservationId; import org.apache.hadoop.yarn.api.records.ReservationId;
import org.apache.hadoop.yarn.api.records.ReservationRequest;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
/** /**
@ -71,7 +70,7 @@ public interface ReservationAllocation extends
* @return the allocationRequests the map of resources requested against the * @return the allocationRequests the map of resources requested against the
* time interval for which they were * time interval for which they were
*/ */
public Map<ReservationInterval, ReservationRequest> getAllocationRequests(); public Map<ReservationInterval, Resource> getAllocationRequests();
/** /**
* Return a string identifying the plan to which the reservation belongs * Return a string identifying the plan to which the reservation belongs

View File

@ -0,0 +1,51 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.reservation;
import org.apache.hadoop.yarn.api.records.ReservationRequest;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.util.resource.Resources;
import java.util.HashMap;
import java.util.Map;
final class ReservationSystemUtil {
private ReservationSystemUtil() {
// not called
}
public static Resource toResource(ReservationRequest request) {
Resource resource = Resources.multiply(request.getCapability(),
(float) request.getNumContainers());
return resource;
}
public static Map<ReservationInterval, Resource> toResources(
Map<ReservationInterval, ReservationRequest> allocations) {
Map<ReservationInterval, Resource> resources =
new HashMap<ReservationInterval, Resource>();
for (Map.Entry<ReservationInterval, ReservationRequest> entry :
allocations.entrySet()) {
resources.put(entry.getKey(),
toResource(entry.getValue()));
}
return resources;
}
}

View File

@ -378,14 +378,15 @@ public static ReservationDefinition generateBigRR(Random rand, long i) {
return rr; return rr;
} }
public static Map<ReservationInterval, ReservationRequest> generateAllocation( public static Map<ReservationInterval, Resource> generateAllocation(
long startTime, long step, int[] alloc) { long startTime, long step, int[] alloc) {
Map<ReservationInterval, ReservationRequest> req = Map<ReservationInterval, Resource> req =
new TreeMap<ReservationInterval, ReservationRequest>(); new TreeMap<ReservationInterval, Resource>();
for (int i = 0; i < alloc.length; i++) { for (int i = 0; i < alloc.length; i++) {
req.put(new ReservationInterval(startTime + i * step, startTime + (i + 1) req.put(new ReservationInterval(startTime + i * step, startTime + (i + 1)
* step), ReservationRequest.newInstance( * step), ReservationSystemUtil.toResource(ReservationRequest
Resource.newInstance(1024, 1), alloc[i])); .newInstance(
Resource.newInstance(1024, 1), alloc[i])));
} }
return req; return req;
} }

View File

@ -19,7 +19,6 @@
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException; import java.io.IOException;
import java.util.Map; import java.util.Map;
@ -198,12 +197,14 @@ public void testInstFailBySum() throws IOException, PlanningException {
@Test(expected = PlanningQuotaException.class) @Test(expected = PlanningQuotaException.class)
public void testFailAvg() throws IOException, PlanningException { public void testFailAvg() throws IOException, PlanningException {
// generate an allocation which violates the 25% average single-shot // generate an allocation which violates the 25% average single-shot
Map<ReservationInterval, ReservationRequest> req = Map<ReservationInterval, Resource> req =
new TreeMap<ReservationInterval, ReservationRequest>(); new TreeMap<ReservationInterval, Resource>();
long win = timeWindow / 2 + 100; long win = timeWindow / 2 + 100;
int cont = (int) Math.ceil(0.5 * totCont); int cont = (int) Math.ceil(0.5 * totCont);
req.put(new ReservationInterval(initTime, initTime + win), req.put(new ReservationInterval(initTime, initTime + win),
ReservationRequest.newInstance(Resource.newInstance(1024, 1), cont)); ReservationSystemUtil.toResource(
ReservationRequest.newInstance(Resource.newInstance(1024, 1),
cont)));
assertTrue(plan.toString(), assertTrue(plan.toString(),
plan.addReservation(new InMemoryReservationAllocation( plan.addReservation(new InMemoryReservationAllocation(
@ -214,12 +215,13 @@ public void testFailAvg() throws IOException, PlanningException {
@Test @Test
public void testFailAvgBySum() throws IOException, PlanningException { public void testFailAvgBySum() throws IOException, PlanningException {
// generate an allocation which violates the 25% average by sum // generate an allocation which violates the 25% average by sum
Map<ReservationInterval, ReservationRequest> req = Map<ReservationInterval, Resource> req =
new TreeMap<ReservationInterval, ReservationRequest>(); new TreeMap<ReservationInterval, Resource>();
long win = 86400000 / 4 + 1; long win = 86400000 / 4 + 1;
int cont = (int) Math.ceil(0.5 * totCont); int cont = (int) Math.ceil(0.5 * totCont);
req.put(new ReservationInterval(initTime, initTime + win), req.put(new ReservationInterval(initTime, initTime + win),
ReservationRequest.newInstance(Resource.newInstance(1024, 1), cont)); ReservationSystemUtil.toResource(ReservationRequest.newInstance(Resource
.newInstance(1024, 1), cont)));
assertTrue(plan.toString(), assertTrue(plan.toString(),
plan.addReservation(new InMemoryReservationAllocation( plan.addReservation(new InMemoryReservationAllocation(
ReservationSystemTestUtil.getNewReservationId(), null, "u1", ReservationSystemTestUtil.getNewReservationId(), null, "u1",

View File

@ -516,7 +516,7 @@ private void prepareBasicPlan() throws PlanningException {
.generateAllocation(0, step, f), res, minAlloc))); .generateAllocation(0, step, f), res, minAlloc)));
int[] f2 = { 5, 5, 5, 5, 5, 5, 5 }; int[] f2 = { 5, 5, 5, 5, 5, 5, 5 };
Map<ReservationInterval, ReservationRequest> alloc = Map<ReservationInterval, Resource> alloc =
ReservationSystemTestUtil.generateAllocation(5000, step, f2); ReservationSystemTestUtil.generateAllocation(5000, step, f2);
assertTrue(plan.toString(), assertTrue(plan.toString(),
plan.addReservation(new InMemoryReservationAllocation( plan.addReservation(new InMemoryReservationAllocation(

View File

@ -100,9 +100,11 @@ public void testAddReservation() {
ReservationDefinition rDef = ReservationDefinition rDef =
createSimpleReservationDefinition(start, start + alloc.length, createSimpleReservationDefinition(start, start + alloc.length,
alloc.length, allocations.values()); alloc.length, allocations.values());
Map<ReservationInterval, Resource> allocs =
ReservationSystemUtil.toResources(allocations);
ReservationAllocation rAllocation = ReservationAllocation rAllocation =
new InMemoryReservationAllocation(reservationID, rDef, user, planName, new InMemoryReservationAllocation(reservationID, rDef, user, planName,
start, start + alloc.length, allocations, resCalc, minAlloc); start, start + alloc.length, allocs, resCalc, minAlloc);
Assert.assertNull(plan.getReservationById(reservationID)); Assert.assertNull(plan.getReservationById(reservationID));
try { try {
plan.addReservation(rAllocation); plan.addReservation(rAllocation);
@ -132,9 +134,11 @@ public void testAddEmptyReservation() {
ReservationDefinition rDef = ReservationDefinition rDef =
createSimpleReservationDefinition(start, start + alloc.length, createSimpleReservationDefinition(start, start + alloc.length,
alloc.length, allocations.values()); alloc.length, allocations.values());
Map<ReservationInterval, Resource> allocs = ReservationSystemUtil.toResources
(allocations);
ReservationAllocation rAllocation = ReservationAllocation rAllocation =
new InMemoryReservationAllocation(reservationID, rDef, user, planName, new InMemoryReservationAllocation(reservationID, rDef, user, planName,
start, start + alloc.length, allocations, resCalc, minAlloc); start, start + alloc.length, allocs, resCalc, minAlloc);
Assert.assertNull(plan.getReservationById(reservationID)); Assert.assertNull(plan.getReservationById(reservationID));
try { try {
plan.addReservation(rAllocation); plan.addReservation(rAllocation);
@ -158,9 +162,11 @@ public void testAddReservationAlreadyExists() {
ReservationDefinition rDef = ReservationDefinition rDef =
createSimpleReservationDefinition(start, start + alloc.length, createSimpleReservationDefinition(start, start + alloc.length,
alloc.length, allocations.values()); alloc.length, allocations.values());
Map<ReservationInterval, Resource> allocs = ReservationSystemUtil.toResources
(allocations);
ReservationAllocation rAllocation = ReservationAllocation rAllocation =
new InMemoryReservationAllocation(reservationID, rDef, user, planName, new InMemoryReservationAllocation(reservationID, rDef, user, planName,
start, start + alloc.length, allocations, resCalc, minAlloc); start, start + alloc.length, allocs, resCalc, minAlloc);
Assert.assertNull(plan.getReservationById(reservationID)); Assert.assertNull(plan.getReservationById(reservationID));
try { try {
plan.addReservation(rAllocation); plan.addReservation(rAllocation);
@ -202,9 +208,11 @@ public void testUpdateReservation() {
ReservationDefinition rDef = ReservationDefinition rDef =
createSimpleReservationDefinition(start, start + alloc.length, createSimpleReservationDefinition(start, start + alloc.length,
alloc.length, allocations.values()); alloc.length, allocations.values());
Map<ReservationInterval, Resource> allocs = ReservationSystemUtil.toResources
(allocations);
ReservationAllocation rAllocation = ReservationAllocation rAllocation =
new InMemoryReservationAllocation(reservationID, rDef, user, planName, new InMemoryReservationAllocation(reservationID, rDef, user, planName,
start, start + alloc.length, allocations, resCalc, minAlloc); start, start + alloc.length, allocs, resCalc, minAlloc);
Assert.assertNull(plan.getReservationById(reservationID)); Assert.assertNull(plan.getReservationById(reservationID));
try { try {
plan.addReservation(rAllocation); plan.addReservation(rAllocation);
@ -226,9 +234,12 @@ public void testUpdateReservation() {
rDef = rDef =
createSimpleReservationDefinition(start, start + updatedAlloc.length, createSimpleReservationDefinition(start, start + updatedAlloc.length,
updatedAlloc.length, allocations.values()); updatedAlloc.length, allocations.values());
Map<ReservationInterval, Resource> updatedAllocs =
ReservationSystemUtil.toResources(allocations);
rAllocation = rAllocation =
new InMemoryReservationAllocation(reservationID, rDef, user, planName, new InMemoryReservationAllocation(reservationID, rDef, user, planName,
start, start + updatedAlloc.length, allocations, resCalc, minAlloc); start, start + updatedAlloc.length, updatedAllocs, resCalc,
minAlloc);
try { try {
plan.updateReservation(rAllocation); plan.updateReservation(rAllocation);
} catch (PlanningException e) { } catch (PlanningException e) {
@ -260,9 +271,11 @@ public void testUpdateNonExistingReservation() {
ReservationDefinition rDef = ReservationDefinition rDef =
createSimpleReservationDefinition(start, start + alloc.length, createSimpleReservationDefinition(start, start + alloc.length,
alloc.length, allocations.values()); alloc.length, allocations.values());
Map<ReservationInterval, Resource> allocs =
ReservationSystemUtil.toResources(allocations);
ReservationAllocation rAllocation = ReservationAllocation rAllocation =
new InMemoryReservationAllocation(reservationID, rDef, user, planName, new InMemoryReservationAllocation(reservationID, rDef, user, planName,
start, start + alloc.length, allocations, resCalc, minAlloc); start, start + alloc.length, allocs, resCalc, minAlloc);
Assert.assertNull(plan.getReservationById(reservationID)); Assert.assertNull(plan.getReservationById(reservationID));
try { try {
plan.updateReservation(rAllocation); plan.updateReservation(rAllocation);
@ -290,9 +303,11 @@ public void testDeleteReservation() {
ReservationDefinition rDef = ReservationDefinition rDef =
createSimpleReservationDefinition(start, start + alloc.length, createSimpleReservationDefinition(start, start + alloc.length,
alloc.length, allocations.values()); alloc.length, allocations.values());
Map<ReservationInterval, Resource> allocs =
ReservationSystemUtil.toResources(allocations);
ReservationAllocation rAllocation = ReservationAllocation rAllocation =
new InMemoryReservationAllocation(reservationID, rDef, user, planName, new InMemoryReservationAllocation(reservationID, rDef, user, planName,
start, start + alloc.length, allocations, resCalc, minAlloc); start, start + alloc.length, allocs, resCalc, minAlloc);
Assert.assertNull(plan.getReservationById(reservationID)); Assert.assertNull(plan.getReservationById(reservationID));
try { try {
plan.addReservation(rAllocation); plan.addReservation(rAllocation);
@ -359,9 +374,11 @@ public void testArchiveCompletedReservations() {
ReservationDefinition rDef1 = ReservationDefinition rDef1 =
createSimpleReservationDefinition(start, start + alloc1.length, createSimpleReservationDefinition(start, start + alloc1.length,
alloc1.length, allocations1.values()); alloc1.length, allocations1.values());
Map<ReservationInterval, Resource> allocs1 =
ReservationSystemUtil.toResources(allocations1);
ReservationAllocation rAllocation = ReservationAllocation rAllocation =
new InMemoryReservationAllocation(reservationID1, rDef1, user, new InMemoryReservationAllocation(reservationID1, rDef1, user,
planName, start, start + alloc1.length, allocations1, resCalc, planName, start, start + alloc1.length, allocs1, resCalc,
minAlloc); minAlloc);
Assert.assertNull(plan.getReservationById(reservationID1)); Assert.assertNull(plan.getReservationById(reservationID1));
try { try {
@ -388,9 +405,11 @@ public void testArchiveCompletedReservations() {
ReservationDefinition rDef2 = ReservationDefinition rDef2 =
createSimpleReservationDefinition(start, start + alloc2.length, createSimpleReservationDefinition(start, start + alloc2.length,
alloc2.length, allocations2.values()); alloc2.length, allocations2.values());
Map<ReservationInterval, Resource> allocs2 =
ReservationSystemUtil.toResources(allocations2);
rAllocation = rAllocation =
new InMemoryReservationAllocation(reservationID2, rDef2, user, new InMemoryReservationAllocation(reservationID2, rDef2, user,
planName, start, start + alloc2.length, allocations2, resCalc, planName, start, start + alloc2.length, allocs2, resCalc,
minAlloc); minAlloc);
Assert.assertNull(plan.getReservationById(reservationID2)); Assert.assertNull(plan.getReservationById(reservationID2));
try { try {

View File

@ -69,7 +69,7 @@ public void testBlocks() {
ReservationDefinition rDef = ReservationDefinition rDef =
createSimpleReservationDefinition(start, start + alloc.length + 1, createSimpleReservationDefinition(start, start + alloc.length + 1,
alloc.length); alloc.length);
Map<ReservationInterval, ReservationRequest> allocations = Map<ReservationInterval, Resource> allocations =
generateAllocation(start, alloc, false, false); generateAllocation(start, alloc, false, false);
ReservationAllocation rAllocation = ReservationAllocation rAllocation =
new InMemoryReservationAllocation(reservationID, rDef, user, planName, new InMemoryReservationAllocation(reservationID, rDef, user, planName,
@ -91,7 +91,7 @@ public void testSteps() {
ReservationDefinition rDef = ReservationDefinition rDef =
createSimpleReservationDefinition(start, start + alloc.length + 1, createSimpleReservationDefinition(start, start + alloc.length + 1,
alloc.length); alloc.length);
Map<ReservationInterval, ReservationRequest> allocations = Map<ReservationInterval, Resource> allocations =
generateAllocation(start, alloc, true, false); generateAllocation(start, alloc, true, false);
ReservationAllocation rAllocation = ReservationAllocation rAllocation =
new InMemoryReservationAllocation(reservationID, rDef, user, planName, new InMemoryReservationAllocation(reservationID, rDef, user, planName,
@ -114,7 +114,7 @@ public void testSkyline() {
ReservationDefinition rDef = ReservationDefinition rDef =
createSimpleReservationDefinition(start, start + alloc.length + 1, createSimpleReservationDefinition(start, start + alloc.length + 1,
alloc.length); alloc.length);
Map<ReservationInterval, ReservationRequest> allocations = Map<ReservationInterval, Resource> allocations =
generateAllocation(start, alloc, true, false); generateAllocation(start, alloc, true, false);
ReservationAllocation rAllocation = ReservationAllocation rAllocation =
new InMemoryReservationAllocation(reservationID, rDef, user, planName, new InMemoryReservationAllocation(reservationID, rDef, user, planName,
@ -137,8 +137,8 @@ public void testZeroAlloaction() {
ReservationDefinition rDef = ReservationDefinition rDef =
createSimpleReservationDefinition(start, start + alloc.length + 1, createSimpleReservationDefinition(start, start + alloc.length + 1,
alloc.length); alloc.length);
Map<ReservationInterval, ReservationRequest> allocations = Map<ReservationInterval, Resource> allocations =
new HashMap<ReservationInterval, ReservationRequest>(); new HashMap<ReservationInterval, Resource>();
ReservationAllocation rAllocation = ReservationAllocation rAllocation =
new InMemoryReservationAllocation(reservationID, rDef, user, planName, new InMemoryReservationAllocation(reservationID, rDef, user, planName,
start, start + alloc.length + 1, allocations, resCalc, minAlloc); start, start + alloc.length + 1, allocations, resCalc, minAlloc);
@ -156,11 +156,13 @@ public void testGangAlloaction() {
ReservationDefinition rDef = ReservationDefinition rDef =
createSimpleReservationDefinition(start, start + alloc.length + 1, createSimpleReservationDefinition(start, start + alloc.length + 1,
alloc.length); alloc.length);
Map<ReservationInterval, ReservationRequest> allocations = boolean isGang = true;
generateAllocation(start, alloc, false, true); Map<ReservationInterval, Resource> allocations =
generateAllocation(start, alloc, false, isGang);
ReservationAllocation rAllocation = ReservationAllocation rAllocation =
new InMemoryReservationAllocation(reservationID, rDef, user, planName, new InMemoryReservationAllocation(reservationID, rDef, user, planName,
start, start + alloc.length + 1, allocations, resCalc, minAlloc); start, start + alloc.length + 1, allocations, resCalc, minAlloc,
isGang);
doAssertions(rAllocation, reservationID, rDef, allocations, start, alloc); doAssertions(rAllocation, reservationID, rDef, allocations, start, alloc);
Assert.assertTrue(rAllocation.containsGangs()); Assert.assertTrue(rAllocation.containsGangs());
for (int i = 0; i < alloc.length; i++) { for (int i = 0; i < alloc.length; i++) {
@ -171,7 +173,7 @@ public void testGangAlloaction() {
private void doAssertions(ReservationAllocation rAllocation, private void doAssertions(ReservationAllocation rAllocation,
ReservationId reservationID, ReservationDefinition rDef, ReservationId reservationID, ReservationDefinition rDef,
Map<ReservationInterval, ReservationRequest> allocations, int start, Map<ReservationInterval, Resource> allocations, int start,
int[] alloc) { int[] alloc) {
Assert.assertEquals(reservationID, rAllocation.getReservationId()); Assert.assertEquals(reservationID, rAllocation.getReservationId());
Assert.assertEquals(rDef, rAllocation.getReservationDefinition()); Assert.assertEquals(rDef, rAllocation.getReservationDefinition());
@ -198,10 +200,10 @@ private ReservationDefinition createSimpleReservationDefinition(long arrival,
return rDef; return rDef;
} }
private Map<ReservationInterval, ReservationRequest> generateAllocation( private Map<ReservationInterval, Resource> generateAllocation(
int startTime, int[] alloc, boolean isStep, boolean isGang) { int startTime, int[] alloc, boolean isStep, boolean isGang) {
Map<ReservationInterval, ReservationRequest> req = Map<ReservationInterval, Resource> req =
new HashMap<ReservationInterval, ReservationRequest>(); new HashMap<ReservationInterval, Resource>();
int numContainers = 0; int numContainers = 0;
for (int i = 0; i < alloc.length; i++) { for (int i = 0; i < alloc.length; i++) {
if (isStep) { if (isStep) {
@ -215,7 +217,8 @@ private Map<ReservationInterval, ReservationRequest> generateAllocation(
if (isGang) { if (isGang) {
rr.setConcurrency(numContainers); rr.setConcurrency(numContainers);
} }
req.put(new ReservationInterval(startTime + i, startTime + i + 1), rr); req.put(new ReservationInterval(startTime + i, startTime + i + 1),
ReservationSystemUtil.toResource(rr));
} }
return req; return req;
} }

View File

@ -46,9 +46,9 @@ public void testBlocks() {
new RLESparseResourceAllocation(resCalc, minAlloc); new RLESparseResourceAllocation(resCalc, minAlloc);
int[] alloc = { 10, 10, 10, 10, 10, 10 }; int[] alloc = { 10, 10, 10, 10, 10, 10 };
int start = 100; int start = 100;
Set<Entry<ReservationInterval, ReservationRequest>> inputs = Set<Entry<ReservationInterval, Resource>> inputs =
generateAllocation(start, alloc, false).entrySet(); generateAllocation(start, alloc, false).entrySet();
for (Entry<ReservationInterval, ReservationRequest> ip : inputs) { for (Entry<ReservationInterval, Resource> ip : inputs) {
rleSparseVector.addInterval(ip.getKey(), ip.getValue()); rleSparseVector.addInterval(ip.getKey(), ip.getValue());
} }
LOG.info(rleSparseVector.toString()); LOG.info(rleSparseVector.toString());
@ -63,7 +63,7 @@ public void testBlocks() {
} }
Assert.assertEquals(Resource.newInstance(0, 0), Assert.assertEquals(Resource.newInstance(0, 0),
rleSparseVector.getCapacityAtTime(start + alloc.length + 2)); rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
for (Entry<ReservationInterval, ReservationRequest> ip : inputs) { for (Entry<ReservationInterval, Resource> ip : inputs) {
rleSparseVector.removeInterval(ip.getKey(), ip.getValue()); rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
} }
LOG.info(rleSparseVector.toString()); LOG.info(rleSparseVector.toString());
@ -83,9 +83,9 @@ public void testSteps() {
new RLESparseResourceAllocation(resCalc, minAlloc); new RLESparseResourceAllocation(resCalc, minAlloc);
int[] alloc = { 10, 10, 10, 10, 10, 10 }; int[] alloc = { 10, 10, 10, 10, 10, 10 };
int start = 100; int start = 100;
Set<Entry<ReservationInterval, ReservationRequest>> inputs = Set<Entry<ReservationInterval, Resource>> inputs =
generateAllocation(start, alloc, true).entrySet(); generateAllocation(start, alloc, true).entrySet();
for (Entry<ReservationInterval, ReservationRequest> ip : inputs) { for (Entry<ReservationInterval, Resource> ip : inputs) {
rleSparseVector.addInterval(ip.getKey(), ip.getValue()); rleSparseVector.addInterval(ip.getKey(), ip.getValue());
} }
LOG.info(rleSparseVector.toString()); LOG.info(rleSparseVector.toString());
@ -101,8 +101,8 @@ public void testSteps() {
} }
Assert.assertEquals(Resource.newInstance(0, 0), Assert.assertEquals(Resource.newInstance(0, 0),
rleSparseVector.getCapacityAtTime(start + alloc.length + 2)); rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
for (Entry<ReservationInterval, ReservationRequest> ip : inputs) { for (Entry<ReservationInterval, Resource> ip : inputs) {
rleSparseVector.removeInterval(ip.getKey(), ip.getValue()); rleSparseVector.removeInterval(ip.getKey(),ip.getValue());
} }
LOG.info(rleSparseVector.toString()); LOG.info(rleSparseVector.toString());
for (int i = 0; i < alloc.length; i++) { for (int i = 0; i < alloc.length; i++) {
@ -121,9 +121,9 @@ public void testSkyline() {
new RLESparseResourceAllocation(resCalc, minAlloc); new RLESparseResourceAllocation(resCalc, minAlloc);
int[] alloc = { 0, 5, 10, 10, 5, 0 }; int[] alloc = { 0, 5, 10, 10, 5, 0 };
int start = 100; int start = 100;
Set<Entry<ReservationInterval, ReservationRequest>> inputs = Set<Entry<ReservationInterval, Resource>> inputs =
generateAllocation(start, alloc, true).entrySet(); generateAllocation(start, alloc, true).entrySet();
for (Entry<ReservationInterval, ReservationRequest> ip : inputs) { for (Entry<ReservationInterval, Resource> ip : inputs) {
rleSparseVector.addInterval(ip.getKey(), ip.getValue()); rleSparseVector.addInterval(ip.getKey(), ip.getValue());
} }
LOG.info(rleSparseVector.toString()); LOG.info(rleSparseVector.toString());
@ -139,7 +139,7 @@ public void testSkyline() {
} }
Assert.assertEquals(Resource.newInstance(0, 0), Assert.assertEquals(Resource.newInstance(0, 0),
rleSparseVector.getCapacityAtTime(start + alloc.length + 2)); rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
for (Entry<ReservationInterval, ReservationRequest> ip : inputs) { for (Entry<ReservationInterval, Resource> ip : inputs) {
rleSparseVector.removeInterval(ip.getKey(), ip.getValue()); rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
} }
LOG.info(rleSparseVector.toString()); LOG.info(rleSparseVector.toString());
@ -157,17 +157,17 @@ public void testZeroAlloaction() {
RLESparseResourceAllocation rleSparseVector = RLESparseResourceAllocation rleSparseVector =
new RLESparseResourceAllocation(resCalc, minAlloc); new RLESparseResourceAllocation(resCalc, minAlloc);
rleSparseVector.addInterval(new ReservationInterval(0, Long.MAX_VALUE), rleSparseVector.addInterval(new ReservationInterval(0, Long.MAX_VALUE),
ReservationRequest.newInstance(Resource.newInstance(0, 0), (0))); Resource.newInstance(0, 0));
LOG.info(rleSparseVector.toString()); LOG.info(rleSparseVector.toString());
Assert.assertEquals(Resource.newInstance(0, 0), Assert.assertEquals(Resource.newInstance(0, 0),
rleSparseVector.getCapacityAtTime(new Random().nextLong())); rleSparseVector.getCapacityAtTime(new Random().nextLong()));
Assert.assertTrue(rleSparseVector.isEmpty()); Assert.assertTrue(rleSparseVector.isEmpty());
} }
private Map<ReservationInterval, ReservationRequest> generateAllocation( private Map<ReservationInterval, Resource> generateAllocation(
int startTime, int[] alloc, boolean isStep) { int startTime, int[] alloc, boolean isStep) {
Map<ReservationInterval, ReservationRequest> req = Map<ReservationInterval, Resource> req =
new HashMap<ReservationInterval, ReservationRequest>(); new HashMap<ReservationInterval, Resource>();
int numContainers = 0; int numContainers = 0;
for (int i = 0; i < alloc.length; i++) { for (int i = 0; i < alloc.length; i++) {
if (isStep) { if (isStep) {
@ -176,9 +176,8 @@ private Map<ReservationInterval, ReservationRequest> generateAllocation(
numContainers = alloc[i]; numContainers = alloc[i];
} }
req.put(new ReservationInterval(startTime + i, startTime + i + 1), req.put(new ReservationInterval(startTime + i, startTime + i + 1),
ReservationSystemUtil.toResource(ReservationRequest.newInstance(
ReservationRequest.newInstance(Resource.newInstance(1024, 1), Resource.newInstance(1024, 1), (numContainers))));
(numContainers)));
} }
return req; return req;
} }

View File

@ -146,14 +146,15 @@ public void testReplanningPlanCapacityLoss() throws PlanningException {
} }
} }
private Map<ReservationInterval, ReservationRequest> generateAllocation( private Map<ReservationInterval, Resource> generateAllocation(
int startTime, int[] alloc) { int startTime, int[] alloc) {
Map<ReservationInterval, ReservationRequest> req = Map<ReservationInterval, Resource> req =
new TreeMap<ReservationInterval, ReservationRequest>(); new TreeMap<ReservationInterval, Resource>();
for (int i = 0; i < alloc.length; i++) { for (int i = 0; i < alloc.length; i++) {
req.put(new ReservationInterval(startTime + i, startTime + i + 1), req.put(new ReservationInterval(startTime + i, startTime + i + 1),
ReservationRequest.newInstance(Resource.newInstance(1024, 1), ReservationSystemUtil.toResource(
alloc[i])); ReservationRequest.newInstance(Resource.newInstance(1024, 1),
alloc[i])));
} }
return req; return req;
} }