YARN-5328. Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

This commit is contained in:
Subru Krishnan 2017-09-06 16:46:01 -07:00
parent e3345e985b
commit b6e7d13696
26 changed files with 1339 additions and 775 deletions

View File

@ -262,6 +262,12 @@ private static void addDeprecatedKeys() {
public static final long DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP =
1000L;
/** The maximum periodicity for the Reservation System. */
public static final String RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
RM_PREFIX + "reservation-system.max-periodicity";
public static final long DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
86400000L;
/**
* Enable periodic monitor threads.
* @see #RM_SCHEDULER_MONITOR_POLICIES

View File

@ -33,7 +33,7 @@
*/
public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
@SuppressWarnings("deprecation")
@SuppressWarnings({"deprecation", "methodlength"})
@Override
public void initializeMemberVariables() {
xmlFilename = new String("yarn-default.xml");
@ -69,6 +69,8 @@ public void initializeMemberVariables() {
configurationPropsToSkipCompare.add(YarnConfiguration
.YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL);
configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
configurationPropsToSkipCompare
.add(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
// Federation default configs to be ignored
configurationPropsToSkipCompare

View File

@ -18,6 +18,17 @@
package org.apache.hadoop.yarn.server.resourcemanager.reservation;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.conf.Configuration;
@ -46,17 +57,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
/**
* This is the implementation of {@link ReservationSystem} based on the
* {@link ResourceScheduler}
@ -66,8 +66,8 @@
public abstract class AbstractReservationSystem extends AbstractService
implements ReservationSystem {
private static final Logger LOG = LoggerFactory
.getLogger(AbstractReservationSystem.class);
private static final Logger LOG =
LoggerFactory.getLogger(AbstractReservationSystem.class);
// private static final String DEFAULT_CAPACITY_SCHEDULER_PLAN
@ -103,6 +103,8 @@ public abstract class AbstractReservationSystem extends AbstractService
private boolean isRecoveryEnabled = false;
private long maxPeriodicity;
/**
* Construct the service.
*
@ -143,36 +145,41 @@ private void initialize(Configuration conf) throws YarnException {
this.conf = conf;
scheduler = rmContext.getScheduler();
// Get the plan step size
planStepSize =
conf.getTimeDuration(
YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
TimeUnit.MILLISECONDS);
planStepSize = conf.getTimeDuration(
YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
TimeUnit.MILLISECONDS);
if (planStepSize < 0) {
planStepSize =
YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP;
}
maxPeriodicity =
conf.getLong(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
if (maxPeriodicity <= 0) {
maxPeriodicity =
YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY;
}
// Create a plan corresponding to every reservable queue
Set<String> planQueueNames = scheduler.getPlanQueues();
for (String planQueueName : planQueueNames) {
Plan plan = initializePlan(planQueueName);
plans.put(planQueueName, plan);
}
isRecoveryEnabled = conf.getBoolean(
YarnConfiguration.RECOVERY_ENABLED,
isRecoveryEnabled = conf.getBoolean(YarnConfiguration.RECOVERY_ENABLED,
YarnConfiguration.DEFAULT_RM_RECOVERY_ENABLED);
if (conf.getBoolean(YarnConfiguration.YARN_RESERVATION_ACL_ENABLE,
YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE) &&
conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE)
&& conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
reservationsACLsManager = new ReservationsACLsManager(scheduler, conf);
}
}
private void loadPlan(String planName,
Map<ReservationId, ReservationAllocationStateProto> reservations)
throws PlanningException {
throws PlanningException {
Plan plan = plans.get(planName);
Resource minAllocation = getMinAllocation();
ResourceCalculator rescCalculator = getResourceCalculator();
@ -248,8 +255,8 @@ private PlanFollower createPlanFollower() {
Class<?> planFollowerPolicyClazz =
conf.getClassByName(planFollowerPolicyClassName);
if (PlanFollower.class.isAssignableFrom(planFollowerPolicyClazz)) {
return (PlanFollower) ReflectionUtils.newInstance(
planFollowerPolicyClazz, conf);
return (PlanFollower) ReflectionUtils
.newInstance(planFollowerPolicyClazz, conf);
} else {
throw new YarnRuntimeException("Class: " + planFollowerPolicyClassName
+ " not instance of " + PlanFollower.class.getCanonicalName());
@ -257,7 +264,8 @@ private PlanFollower createPlanFollower() {
} catch (ClassNotFoundException e) {
throw new YarnRuntimeException(
"Could not instantiate PlanFollowerPolicy: "
+ planFollowerPolicyClassName, e);
+ planFollowerPolicyClassName,
e);
}
}
@ -371,9 +379,8 @@ public void setQueueForReservation(ReservationId reservationId,
public ReservationId getNewReservationId() {
writeLock.lock();
try {
ReservationId resId =
ReservationId.newInstance(ResourceManager.getClusterTimeStamp(),
resCounter.incrementAndGet());
ReservationId resId = ReservationId.newInstance(
ResourceManager.getClusterTimeStamp(), resCounter.incrementAndGet());
LOG.info("Allocated new reservationId: " + resId);
return resId;
} finally {
@ -390,8 +397,11 @@ public Map<String, Plan> getAllPlans() {
* Get the default reservation system corresponding to the scheduler
*
* @param scheduler the scheduler for which the reservation system is required
*
* @return the {@link ReservationSystem} based on the configured scheduler
*/
public static String getDefaultReservationSystem(ResourceScheduler scheduler) {
public static String getDefaultReservationSystem(
ResourceScheduler scheduler) {
if (scheduler instanceof CapacityScheduler) {
return CapacityReservationSystem.class.getName();
} else if (scheduler instanceof FairScheduler) {
@ -409,12 +419,11 @@ protected Plan initializePlan(String planQueueName) throws YarnException {
Resource maxAllocation = getMaxAllocation();
ResourceCalculator rescCalc = getResourceCalculator();
Resource totCap = getPlanQueueCapacity(planQueueName);
Plan plan =
new InMemoryPlan(getRootQueueMetrics(), adPolicy,
getAgent(planQueuePath), totCap, planStepSize, rescCalc,
minAllocation, maxAllocation, planQueueName,
getReplanner(planQueuePath), getReservationSchedulerConfiguration()
.getMoveOnExpiry(planQueuePath), rmContext);
Plan plan = new InMemoryPlan(getRootQueueMetrics(), adPolicy,
getAgent(planQueuePath), totCap, planStepSize, rescCalc, minAllocation,
maxAllocation, planQueueName, getReplanner(planQueuePath),
getReservationSchedulerConfiguration().getMoveOnExpiry(planQueuePath),
maxPeriodicity, rmContext);
LOG.info("Initialized plan {} based on reservable queue {}",
plan.toString(), planQueueName);
return plan;
@ -477,8 +486,8 @@ protected SharingPolicy getAdmissionPolicy(String queueName) {
Class<?> admissionPolicyClazz =
conf.getClassByName(admissionPolicyClassName);
if (SharingPolicy.class.isAssignableFrom(admissionPolicyClazz)) {
return (SharingPolicy) ReflectionUtils.newInstance(
admissionPolicyClazz, conf);
return (SharingPolicy) ReflectionUtils.newInstance(admissionPolicyClazz,
conf);
} else {
throw new YarnRuntimeException("Class: " + admissionPolicyClassName
+ " not instance of " + SharingPolicy.class.getCanonicalName());
@ -493,8 +502,7 @@ public ReservationsACLsManager getReservationsACLsManager() {
return this.reservationsACLsManager;
}
protected abstract ReservationSchedulerConfiguration
getReservationSchedulerConfiguration();
protected abstract ReservationSchedulerConfiguration getReservationSchedulerConfiguration();
protected abstract String getPlanQueuePath(String planQueueName);

View File

@ -18,6 +18,14 @@
package org.apache.hadoop.yarn.server.resourcemanager.reservation;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ReservationId;
import org.apache.hadoop.yarn.api.records.Resource;
@ -33,24 +41,17 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
private static final Logger LOG = LoggerFactory
.getLogger(AbstractSchedulerPlanFollower.class);
private static final Logger LOG =
LoggerFactory.getLogger(AbstractSchedulerPlanFollower.class);
protected Collection<Plan> plans = new ArrayList<Plan>();
protected YarnScheduler scheduler;
protected Clock clock;
@Override
public void init(Clock clock, ResourceScheduler sched, Collection<Plan> plans) {
public void init(Clock clock, ResourceScheduler sched,
Collection<Plan> plans) {
this.clock = clock;
this.scheduler = sched;
this.plans.addAll(plans);
@ -71,7 +72,7 @@ public synchronized void setPlans(Collection<Plan> plans) {
@Override
public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
String planQueueName = plan.getQueueName();
String planQueueName = plan.getQueueName();
if (LOG.isDebugEnabled()) {
LOG.debug("Running plan follower edit policy for plan: " + planQueueName);
}
@ -82,12 +83,14 @@ public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
now += step - (now % step);
}
Queue planQueue = getPlanQueue(planQueueName);
if (planQueue == null) return;
if (planQueue == null) {
return;
}
// first we publish to the plan the current availability of resources
Resource clusterResources = scheduler.getClusterResource();
Resource planResources = getPlanResources(plan, planQueue,
clusterResources);
Resource planResources =
getPlanResources(plan, planQueue, clusterResources);
Set<ReservationAllocation> currentReservations =
plan.getReservationsAtTime(now);
Set<String> curReservationNames = new HashSet<String>();
@ -95,12 +98,11 @@ public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
int numRes = getReservedResources(now, currentReservations,
curReservationNames, reservedResources);
// create the default reservation queue if it doesnt exist
String defReservationId = getReservationIdFromQueueName(planQueueName) +
ReservationConstants.DEFAULT_QUEUE_SUFFIX;
String defReservationQueue = getReservationQueueName(planQueueName,
defReservationId);
createDefaultReservationQueue(planQueueName, planQueue,
defReservationId);
String defReservationId = getReservationIdFromQueueName(planQueueName)
+ ReservationConstants.DEFAULT_QUEUE_SUFFIX;
String defReservationQueue =
getReservationQueueName(planQueueName, defReservationId);
createDefaultReservationQueue(planQueueName, planQueue, defReservationId);
curReservationNames.add(defReservationId);
// if the resources dedicated to this plan has shrunk invoke replanner
boolean shouldResize = false;
@ -149,10 +151,8 @@ public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
// sort allocations from the one giving up the most resources, to the
// one asking for the most avoid order-of-operation errors that
// temporarily violate 100% capacity bound
List<ReservationAllocation> sortedAllocations =
sortByDelta(
new ArrayList<ReservationAllocation>(currentReservations), now,
plan);
List<ReservationAllocation> sortedAllocations = sortByDelta(
new ArrayList<ReservationAllocation>(currentReservations), now, plan);
for (ReservationAllocation res : sortedAllocations) {
String currResId = res.getReservationId().toString();
if (curReservationNames.contains(currResId)) {
@ -163,10 +163,9 @@ public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
if (planResources.getMemorySize() > 0
&& planResources.getVirtualCores() > 0) {
if (shouldResize) {
capToAssign =
calculateReservationToPlanProportion(
plan.getResourceCalculator(), planResources,
reservedResources, capToAssign);
capToAssign = calculateReservationToPlanProportion(
plan.getResourceCalculator(), planResources, reservedResources,
capToAssign);
}
targetCapacity =
calculateReservationToPlanRatio(plan.getResourceCalculator(),
@ -185,7 +184,8 @@ public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
maxCapacity = targetCapacity;
}
try {
setQueueEntitlement(planQueueName, currResId, targetCapacity, maxCapacity);
setQueueEntitlement(planQueueName, currResId, targetCapacity,
maxCapacity);
} catch (YarnException e) {
LOG.warn("Exception while trying to size reservation for plan: {}",
currResId, planQueueName, e);
@ -196,9 +196,10 @@ public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
// compute the default queue capacity
float defQCap = 1.0f - totalAssignedCapacity;
if (LOG.isDebugEnabled()) {
LOG.debug("PlanFollowerEditPolicyTask: total Plan Capacity: {} "
+ "currReservation: {} default-queue capacity: {}", planResources,
numRes, defQCap);
LOG.debug(
"PlanFollowerEditPolicyTask: total Plan Capacity: {} "
+ "currReservation: {} default-queue capacity: {}",
planResources, numRes, defQCap);
}
// set the default queue to eat-up all remaining capacity
try {
@ -225,12 +226,11 @@ protected String getReservationIdFromQueueName(String resQueueName) {
}
protected void setQueueEntitlement(String planQueueName, String currResId,
float targetCapacity,
float maxCapacity) throws YarnException {
String reservationQueueName = getReservationQueueName(planQueueName,
currResId);
scheduler.setEntitlement(reservationQueueName, new QueueEntitlement(
targetCapacity, maxCapacity));
float targetCapacity, float maxCapacity) throws YarnException {
String reservationQueueName =
getReservationQueueName(planQueueName, currResId);
scheduler.setEntitlement(reservationQueueName,
new QueueEntitlement(targetCapacity, maxCapacity));
}
// Schedulers have different ways of naming queues. See YARN-2773
@ -244,14 +244,21 @@ protected String getReservationQueueName(String planQueueName,
* Then move all apps in the set of queues to the parent plan queue's default
* reservation queue if move is enabled. Finally cleanups the queue by killing
* any apps (if move is disabled or move failed) and removing the queue
*
* @param planQueueName the name of {@code PlanQueue}
* @param shouldMove flag to indicate if any running apps should be moved or
* killed
* @param toRemove the remnant apps to clean up
* @param defReservationQueue the default {@code ReservationQueue} of the
* {@link Plan}
*/
protected void cleanupExpiredQueues(String planQueueName,
boolean shouldMove, Set<String> toRemove, String defReservationQueue) {
protected void cleanupExpiredQueues(String planQueueName, boolean shouldMove,
Set<String> toRemove, String defReservationQueue) {
for (String expiredReservationId : toRemove) {
try {
// reduce entitlement to 0
String expiredReservation = getReservationQueueName(planQueueName,
expiredReservationId);
String expiredReservation =
getReservationQueueName(planQueueName, expiredReservationId);
setQueueEntitlement(planQueueName, expiredReservation, 0.0f, 0.0f);
if (shouldMove) {
moveAppsInQueueSync(expiredReservation, defReservationQueue);
@ -275,7 +282,7 @@ protected void cleanupExpiredQueues(String planQueueName,
* reservation queue in a synchronous fashion
*/
private void moveAppsInQueueSync(String expiredReservation,
String defReservationQueue) {
String defReservationQueue) {
List<ApplicationAttemptId> activeApps =
scheduler.getAppsInQueue(expiredReservation);
if (activeApps.isEmpty()) {
@ -287,16 +294,16 @@ private void moveAppsInQueueSync(String expiredReservation,
scheduler.moveApplication(app.getApplicationId(), defReservationQueue);
} catch (YarnException e) {
LOG.warn(
"Encountered unexpected error during migration of application: {}" +
" from reservation: {}",
"Encountered unexpected error during migration of application: {}"
+ " from reservation: {}",
app, expiredReservation, e);
}
}
}
protected int getReservedResources(long now, Set<ReservationAllocation>
currentReservations, Set<String> curReservationNames,
Resource reservedResources) {
protected int getReservedResources(long now,
Set<ReservationAllocation> currentReservations,
Set<String> curReservationNames, Resource reservedResources) {
int numRes = 0;
if (currentReservations != null) {
numRes = currentReservations.size();
@ -312,23 +319,30 @@ protected int getReservedResources(long now, Set<ReservationAllocation>
* Sort in the order from the least new amount of resources asked (likely
* negative) to the highest. This prevents "order-of-operation" errors related
* to exceeding 100% capacity temporarily.
*
* @param currentReservations the currently active reservations
* @param now the current time
* @param plan the {@link Plan} that is being considered
*
* @return the sorted list of {@link ReservationAllocation}s
*/
protected List<ReservationAllocation> sortByDelta(
List<ReservationAllocation> currentReservations, long now, Plan plan) {
Collections.sort(currentReservations, new ReservationAllocationComparator(
now, this, plan));
Collections.sort(currentReservations,
new ReservationAllocationComparator(now, this, plan));
return currentReservations;
}
/**
* Get queue associated with reservable queue named
* @param planQueueName Name of the reservable queue
* Get queue associated with reservable queue named.
*
* @param planQueueName name of the reservable queue
* @return queue associated with the reservable queue
*/
protected abstract Queue getPlanQueue(String planQueueName);
/**
* Resizes reservations based on currently available resources
* Resizes reservations based on currently available resources.
*/
private Resource calculateReservationToPlanProportion(
ResourceCalculator rescCalculator, Resource availablePlanResources,
@ -338,7 +352,7 @@ private Resource calculateReservationToPlanProportion(
}
/**
* Calculates ratio of reservationResources to planResources
* Calculates ratio of reservationResources to planResources.
*/
private float calculateReservationToPlanRatio(
ResourceCalculator rescCalculator, Resource clusterResources,
@ -348,7 +362,7 @@ private float calculateReservationToPlanRatio(
}
/**
* Check if plan resources are less than expected reservation resources
* Check if plan resources are less than expected reservation resources.
*/
private boolean arePlanResourcesLessThanReservations(
ResourceCalculator rescCalculator, Resource clusterResources,
@ -358,38 +372,56 @@ private boolean arePlanResourcesLessThanReservations(
}
/**
* Get a list of reservation queues for this planQueue
* Get a list of reservation queues for this planQueue.
*
* @param planQueue the queue for the current {@link Plan}
*
* @return the queues corresponding to the reservations
*/
protected abstract List<? extends Queue> getChildReservationQueues(
Queue planQueue);
/**
* Add a new reservation queue for reservation currResId for this planQueue
* Add a new reservation queue for reservation currResId for this planQueue.
*/
protected abstract void addReservationQueue(
String planQueueName, Queue queue, String currResId);
protected abstract void addReservationQueue(String planQueueName, Queue queue,
String currResId);
/**
* Creates the default reservation queue for use when no reservation is
* used for applications submitted to this planQueue
* Creates the default reservation queue for use when no reservation is used
* for applications submitted to this planQueue.
*
* @param planQueueName name of the reservable queue
* @param queue the queue for the current {@link Plan}
* @param defReservationQueue name of the default {@code ReservationQueue}
*/
protected abstract void createDefaultReservationQueue(
String planQueueName, Queue queue, String defReservationQueue);
protected abstract void createDefaultReservationQueue(String planQueueName,
Queue queue, String defReservationQueue);
/**
* Get plan resources for this planQueue
* Get plan resources for this planQueue.
*
* @param plan the current {@link Plan} being considered
* @param clusterResources the resources available in the cluster
*
* @return the resources allocated to the specified {@link Plan}
*/
protected abstract Resource getPlanResources(
Plan plan, Queue queue, Resource clusterResources);
protected abstract Resource getPlanResources(Plan plan, Queue queue,
Resource clusterResources);
/**
* Get reservation queue resources if it exists otherwise return null.
*
* @param plan the current {@link Plan} being considered
* @param reservationId the identifier of the reservation
*
* @return the resources allocated to the specified reservation
*/
protected abstract Resource getReservationQueueResourceIfExists(Plan plan,
ReservationId reservationId);
private static class ReservationAllocationComparator implements
Comparator<ReservationAllocation> {
private static class ReservationAllocationComparator
implements Comparator<ReservationAllocation> {
AbstractSchedulerPlanFollower planFollower;
long now;
Plan plan;
@ -404,14 +436,12 @@ private static class ReservationAllocationComparator implements
private Resource getUnallocatedReservedResources(
ReservationAllocation reservation) {
Resource resResource;
Resource reservationResource = planFollower
.getReservationQueueResourceIfExists
(plan, reservation.getReservationId());
Resource reservationResource =
planFollower.getReservationQueueResourceIfExists(plan,
reservation.getReservationId());
if (reservationResource != null) {
resResource =
Resources.subtract(
reservation.getResourcesAtTime(now),
reservationResource);
resResource = Resources.subtract(reservation.getResourcesAtTime(now),
reservationResource);
} else {
resResource = reservation.getResourcesAtTime(now);
}
@ -428,4 +458,3 @@ public int compare(ReservationAllocation lhs, ReservationAllocation rhs) {
}
}
}

View File

@ -6,9 +6,9 @@
* 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
*
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.
@ -33,9 +33,10 @@
import org.apache.hadoop.yarn.api.records.ReservationId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planner;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
@ -64,9 +65,14 @@ public class InMemoryPlan implements Plan {
private RLESparseResourceAllocation rleSparseVector;
private PeriodicRLESparseResourceAllocation periodicRle;
private Map<String, RLESparseResourceAllocation> userResourceAlloc =
new HashMap<String, RLESparseResourceAllocation>();
private Map<String, RLESparseResourceAllocation> userPeriodicResourceAlloc =
new HashMap<String, RLESparseResourceAllocation>();
private Map<String, RLESparseResourceAllocation> userActiveReservationCount =
new HashMap<String, RLESparseResourceAllocation>();
@ -96,15 +102,27 @@ public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
String queueName, Planner replanner, boolean getMoveOnExpiry,
RMContext rmContext) {
this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
maxAlloc, queueName, replanner, getMoveOnExpiry, rmContext,
new UTCClock());
maxAlloc, queueName, replanner, getMoveOnExpiry,
YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
rmContext);
}
public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
ReservationAgent agent, Resource totalCapacity, long step,
ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
String queueName, Planner replanner, boolean getMoveOnExpiry,
RMContext rmContext, Clock clock) {
long maxPeriodicty, RMContext rmContext) {
this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
maxAlloc, queueName, replanner, getMoveOnExpiry, maxPeriodicty,
rmContext, new UTCClock());
}
@SuppressWarnings("checkstyle:parameternumber")
public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
ReservationAgent agent, Resource totalCapacity, long step,
ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
String queueName, Planner replanner, boolean getMoveOnExpiry,
long maxPeriodicty, RMContext rmContext, Clock clock) {
this.queueMetrics = queueMetrics;
this.policy = policy;
this.agent = agent;
@ -114,6 +132,8 @@ public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
this.minAlloc = minAlloc;
this.maxAlloc = maxAlloc;
this.rleSparseVector = new RLESparseResourceAllocation(resCalc);
this.periodicRle =
new PeriodicRLESparseResourceAllocation(resCalc, maxPeriodicty);
this.queueName = queueName;
this.replanner = replanner;
this.getMoveOnExpiry = getMoveOnExpiry;
@ -126,6 +146,39 @@ public QueueMetrics getQueueMetrics() {
return queueMetrics;
}
private RLESparseResourceAllocation getUserRLEResourceAllocation(String user,
long period) {
RLESparseResourceAllocation resAlloc = null;
if (period > 0) {
if (userPeriodicResourceAlloc.containsKey(user)) {
resAlloc = userPeriodicResourceAlloc.get(user);
} else {
resAlloc = new PeriodicRLESparseResourceAllocation(resCalc,
periodicRle.getTimePeriod());
userPeriodicResourceAlloc.put(user, resAlloc);
}
} else {
if (userResourceAlloc.containsKey(user)) {
resAlloc = userResourceAlloc.get(user);
} else {
resAlloc = new RLESparseResourceAllocation(resCalc);
userResourceAlloc.put(user, resAlloc);
}
}
return resAlloc;
}
private void gcUserRLEResourceAllocation(String user, long period) {
if (period > 0) {
if (userPeriodicResourceAlloc.get(user).isEmpty()) {
userPeriodicResourceAlloc.remove(user);
}
} else {
if (userResourceAlloc.get(user).isEmpty()) {
userResourceAlloc.remove(user);
}
}
}
private void incrementAllocation(ReservationAllocation reservation) {
assert (readWriteLock.isWriteLockedByCurrentThread());
@ -133,11 +186,10 @@ private void incrementAllocation(ReservationAllocation reservation) {
reservation.getAllocationRequests();
// check if we have encountered the user earlier and if not add an entry
String user = reservation.getUser();
RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
if (resAlloc == null) {
resAlloc = new RLESparseResourceAllocation(resCalc);
userResourceAlloc.put(user, resAlloc);
}
long period = reservation.getPeriodicity();
RLESparseResourceAllocation resAlloc =
getUserRLEResourceAllocation(user, period);
RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
if (resCount == null) {
resCount = new RLESparseResourceAllocation(resCalc);
@ -149,14 +201,43 @@ private void incrementAllocation(ReservationAllocation reservation) {
for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
.entrySet()) {
resAlloc.addInterval(r.getKey(), r.getValue());
rleSparseVector.addInterval(r.getKey(), r.getValue());
if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
ZERO_RESOURCE)) {
earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
latestActive = Math.max(latestActive, r.getKey().getEndTime());
if (period > 0L) {
for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
long rStart = r.getKey().getStartTime() + i * period;
long rEnd = r.getKey().getEndTime() + i * period;
// handle wrap-around
if (rEnd > periodicRle.getTimePeriod()) {
long diff = rEnd - periodicRle.getTimePeriod();
rEnd = periodicRle.getTimePeriod();
ReservationInterval newInterval = new ReservationInterval(0, diff);
periodicRle.addInterval(newInterval, r.getValue());
resAlloc.addInterval(newInterval, r.getValue());
}
ReservationInterval newInterval =
new ReservationInterval(rStart, rEnd);
periodicRle.addInterval(newInterval, r.getValue());
resAlloc.addInterval(newInterval, r.getValue());
}
} else {
rleSparseVector.addInterval(r.getKey(), r.getValue());
resAlloc.addInterval(r.getKey(), r.getValue());
if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
ZERO_RESOURCE)) {
earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
latestActive = Math.max(latestActive, r.getKey().getEndTime());
}
}
}
// periodic reservations are active from start time and good till cancelled
if (period > 0L) {
earliestActive = reservation.getStartTime();
latestActive = Long.MAX_VALUE;
}
resCount.addInterval(new ReservationInterval(earliestActive, latestActive),
Resource.newInstance(1, 1));
}
@ -166,27 +247,55 @@ private void decrementAllocation(ReservationAllocation reservation) {
Map<ReservationInterval, Resource> allocationRequests =
reservation.getAllocationRequests();
String user = reservation.getUser();
RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
long period = reservation.getPeriodicity();
RLESparseResourceAllocation resAlloc =
getUserRLEResourceAllocation(user, period);
long earliestActive = Long.MAX_VALUE;
long latestActive = Long.MIN_VALUE;
for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
.entrySet()) {
resAlloc.removeInterval(r.getKey(), r.getValue());
rleSparseVector.removeInterval(r.getKey(), r.getValue());
if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
ZERO_RESOURCE)) {
earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
latestActive = Math.max(latestActive, r.getKey().getEndTime());
if (period > 0L) {
for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
long rStart = r.getKey().getStartTime() + i * period;
long rEnd = r.getKey().getEndTime() + i * period;
// handle wrap-around
if (rEnd > periodicRle.getTimePeriod()) {
long diff = rEnd - periodicRle.getTimePeriod();
rEnd = periodicRle.getTimePeriod();
ReservationInterval newInterval = new ReservationInterval(0, diff);
periodicRle.removeInterval(newInterval, r.getValue());
resAlloc.removeInterval(newInterval, r.getValue());
}
ReservationInterval newInterval =
new ReservationInterval(rStart, rEnd);
periodicRle.removeInterval(newInterval, r.getValue());
resAlloc.removeInterval(newInterval, r.getValue());
}
} else {
rleSparseVector.removeInterval(r.getKey(), r.getValue());
resAlloc.removeInterval(r.getKey(), r.getValue());
if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
ZERO_RESOURCE)) {
earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
latestActive = Math.max(latestActive, r.getKey().getEndTime());
}
}
}
if (resAlloc.isEmpty()) {
userResourceAlloc.remove(user);
}
gcUserRLEResourceAllocation(user, period);
RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
resCount.removeInterval(new ReservationInterval(earliestActive,
latestActive), Resource.newInstance(1, 1));
// periodic reservations are active from start time and good till cancelled
if (period > 0L) {
earliestActive = reservation.getStartTime();
latestActive = Long.MAX_VALUE;
}
resCount.removeInterval(
new ReservationInterval(earliestActive, latestActive),
Resource.newInstance(1, 1));
if (resCount.isEmpty()) {
userActiveReservationCount.remove(user);
}
@ -198,9 +307,9 @@ public Set<ReservationAllocation> getAllReservations() {
if (currentReservations != null) {
Set<ReservationAllocation> flattenedReservations =
new TreeSet<ReservationAllocation>();
for (Set<InMemoryReservationAllocation> reservationEntries :
currentReservations.values()) {
flattenedReservations.addAll(reservationEntries);
for (Set<InMemoryReservationAllocation> res : currentReservations
.values()) {
flattenedReservations.addAll(res);
}
return flattenedReservations;
} else {
@ -218,19 +327,16 @@ public boolean addReservation(ReservationAllocation reservation,
InMemoryReservationAllocation inMemReservation =
(InMemoryReservationAllocation) reservation;
if (inMemReservation.getUser() == null) {
String errMsg =
"The specified Reservation with ID "
+ inMemReservation.getReservationId()
+ " is not mapped to any user";
String errMsg = "The specified Reservation with ID "
+ inMemReservation.getReservationId() + " is not mapped to any user";
LOG.error(errMsg);
throw new IllegalArgumentException(errMsg);
}
writeLock.lock();
try {
if (reservationTable.containsKey(inMemReservation.getReservationId())) {
String errMsg =
"The specified Reservation with ID "
+ inMemReservation.getReservationId() + " already exists";
String errMsg = "The specified Reservation with ID "
+ inMemReservation.getReservationId() + " already exists";
LOG.error(errMsg);
throw new IllegalArgumentException(errMsg);
}
@ -246,9 +352,8 @@ public boolean addReservation(ReservationAllocation reservation,
getQueueName(), inMemReservation.getReservationId().toString());
}
}
ReservationInterval searchInterval =
new ReservationInterval(inMemReservation.getStartTime(),
inMemReservation.getEndTime());
ReservationInterval searchInterval = new ReservationInterval(
inMemReservation.getStartTime(), inMemReservation.getEndTime());
Set<InMemoryReservationAllocation> reservations =
currentReservations.get(searchInterval);
if (reservations == null) {
@ -280,9 +385,8 @@ public boolean updateReservation(ReservationAllocation reservation)
ReservationId resId = reservation.getReservationId();
ReservationAllocation currReservation = getReservationById(resId);
if (currReservation == null) {
String errMsg =
"The specified Reservation with ID " + resId
+ " does not exist in the plan";
String errMsg = "The specified Reservation with ID " + resId
+ " does not exist in the plan";
LOG.error(errMsg);
throw new IllegalArgumentException(errMsg);
}
@ -318,9 +422,8 @@ public boolean updateReservation(ReservationAllocation reservation)
private boolean removeReservation(ReservationAllocation reservation) {
assert (readWriteLock.isWriteLockedByCurrentThread());
ReservationInterval searchInterval =
new ReservationInterval(reservation.getStartTime(),
reservation.getEndTime());
ReservationInterval searchInterval = new ReservationInterval(
reservation.getStartTime(), reservation.getEndTime());
Set<InMemoryReservationAllocation> reservations =
currentReservations.get(searchInterval);
if (reservations != null) {
@ -337,16 +440,15 @@ private boolean removeReservation(ReservationAllocation reservation) {
currentReservations.remove(searchInterval);
}
} else {
String errMsg =
"The specified Reservation with ID " + reservation.getReservationId()
+ " does not exist in the plan";
String errMsg = "The specified Reservation with ID "
+ reservation.getReservationId() + " does not exist in the plan";
LOG.error(errMsg);
throw new IllegalArgumentException(errMsg);
}
reservationTable.remove(reservation.getReservationId());
decrementAllocation(reservation);
LOG.info("Sucessfully deleted reservation: {} in plan.",
reservation.getReservationId());
reservation.getReservationId());
return true;
}
@ -356,9 +458,8 @@ public boolean deleteReservation(ReservationId reservationID) {
try {
ReservationAllocation reservation = getReservationById(reservationID);
if (reservation == null) {
String errMsg =
"The specified Reservation with ID " + reservationID
+ " does not exist in the plan";
String errMsg = "The specified Reservation with ID " + reservationID
+ " does not exist in the plan";
LOG.error(errMsg);
throw new IllegalArgumentException(errMsg);
}
@ -453,66 +554,90 @@ public RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
long start, long end) {
readLock.lock();
try {
// merge periodic and non-periodic allocations
RLESparseResourceAllocation userResAlloc = userResourceAlloc.get(user);
RLESparseResourceAllocation userPeriodicResAlloc =
userPeriodicResourceAlloc.get(user);
if (userResAlloc != null && userPeriodicResAlloc != null) {
return RLESparseResourceAllocation.merge(resCalc, totalCapacity,
userResAlloc, userPeriodicResAlloc, RLEOperator.add, start, end);
}
if (userResAlloc != null) {
return userResAlloc.getRangeOverlapping(start, end);
} else {
return new RLESparseResourceAllocation(resCalc);
}
if (userPeriodicResAlloc != null) {
return userPeriodicResAlloc.getRangeOverlapping(start, end);
}
} catch (PlanningException e) {
LOG.warn("Exception while trying to merge periodic"
+ " and non-periodic user allocations: {}", e.getMessage(), e);
} finally {
readLock.unlock();
}
return new RLESparseResourceAllocation(resCalc);
}
@Override
public Resource getTotalCommittedResources(long t) {
readLock.lock();
try {
return rleSparseVector.getCapacityAtTime(t);
return Resources.add(rleSparseVector.getCapacityAtTime(t),
periodicRle.getCapacityAtTime(t));
} finally {
readLock.unlock();
}
}
@Override
public Set<ReservationAllocation> getReservations(ReservationId
reservationID, ReservationInterval interval) {
public Set<ReservationAllocation> getReservations(ReservationId reservationID,
ReservationInterval interval) {
return getReservations(reservationID, interval, null);
}
@Override
public Set<ReservationAllocation> getReservations(ReservationId
reservationID, ReservationInterval interval, String user) {
public Set<ReservationAllocation> getReservations(ReservationId reservationID,
ReservationInterval interval, String user) {
if (reservationID != null) {
ReservationAllocation allocation = getReservationById(reservationID);
if (allocation == null){
if (allocation == null) {
return Collections.emptySet();
}
return Collections.singleton(allocation);
}
long startTime = interval == null? 0 : interval.getStartTime();
long endTime = interval == null? Long.MAX_VALUE : interval.getEndTime();
long startTime = interval == null ? 0 : interval.getStartTime();
long endTime = interval == null ? Long.MAX_VALUE : interval.getEndTime();
ReservationInterval searchInterval =
new ReservationInterval(endTime, Long.MAX_VALUE);
new ReservationInterval(endTime, Long.MAX_VALUE);
readLock.lock();
try {
SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>>
reservations = currentReservations.headMap(searchInterval, true);
if (!reservations.isEmpty()) {
Set<ReservationAllocation> flattenedReservations =
new HashSet<>();
for (Set<InMemoryReservationAllocation> reservationEntries :
reservations.values()) {
for (InMemoryReservationAllocation res : reservationEntries) {
if (res.getEndTime() > startTime) {
if (user != null && !user.isEmpty()
&& !res.getUser().equals(user)) {
continue;
SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>> res =
currentReservations.headMap(searchInterval, true);
if (!res.isEmpty()) {
Set<ReservationAllocation> flattenedReservations = new HashSet<>();
for (Set<InMemoryReservationAllocation> resEntries : res.values()) {
for (InMemoryReservationAllocation reservation : resEntries) {
// validate user
if (user != null && !user.isEmpty()
&& !reservation.getUser().equals(user)) {
continue;
}
// handle periodic reservations
long period = reservation.getPeriodicity();
if (period > 0) {
long t = endTime % period;
// check for both contained and wrap-around reservations
if ((t - startTime) * (t - endTime)
* (startTime - endTime) >= 0) {
flattenedReservations.add(reservation);
}
} else {
// check for non-periodic reservations
if (reservation.getEndTime() > startTime) {
flattenedReservations.add(reservation);
}
flattenedReservations.add(res);
}
}
}
@ -550,36 +675,82 @@ public Resource getTotalCapacity() {
@Override
public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
ReservationId oldId, long start, long end) throws PlanningException {
ReservationId oldId, long start, long end, long period)
throws PlanningException {
readLock.lock();
try {
// create RLE of totCapacity
TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
totAvailable.put(start, Resources.clone(totalCapacity));
RLESparseResourceAllocation totRLEAvail =
new RLESparseResourceAllocation(totAvailable, resCalc);
// subtract used from available
RLESparseResourceAllocation netAvailable;
// for non-periodic return simple available resources
if (period == 0) {
netAvailable =
RLESparseResourceAllocation.merge(resCalc,
Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
RLEOperator.subtractTestNonNegative, start, end);
// create RLE of totCapacity
TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
totAvailable.put(start, Resources.clone(totalCapacity));
RLESparseResourceAllocation totRLEAvail =
new RLESparseResourceAllocation(totAvailable, resCalc);
// subtract used from available
RLESparseResourceAllocation netAvailable;
netAvailable = RLESparseResourceAllocation.merge(resCalc,
Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
RLEOperator.subtractTestNonNegative, start, end);
// remove periodic component
netAvailable = RLESparseResourceAllocation.merge(resCalc,
Resources.clone(totalCapacity), netAvailable, periodicRle,
RLEOperator.subtractTestNonNegative, start, end);
// add back in old reservation used resources if any
ReservationAllocation old = reservationTable.get(oldId);
if (old != null) {
RLESparseResourceAllocation addBackPrevious =
old.getResourcesOverTime(start, end);
netAvailable = RLESparseResourceAllocation.merge(resCalc,
Resources.clone(totalCapacity), netAvailable, addBackPrevious,
RLEOperator.add, start, end);
}
// lower it if this is needed by the sharing policy
netAvailable = getSharingPolicy().availableResources(netAvailable, this,
user, oldId, start, end);
return netAvailable;
} else {
if (periodicRle.getTimePeriod() % period != 0) {
throw new PlanningException("The reservation periodicity (" + period
+ ") must be" + "an exact divider of the system maxPeriod ("
+ periodicRle.getTimePeriod() + ")");
}
// find the minimum resources available among all the instances that fit
// in the LCM
long numInstInLCM = periodicRle.getTimePeriod() / period;
RLESparseResourceAllocation minOverLCM =
getAvailableResourceOverTime(user, oldId, start, end, 0);
for (int i = 1; i < numInstInLCM; i++) {
long rStart = start + i * period;
long rEnd = end + i * period;
// recursive invocation of non-periodic range (to pick raw-info)
RLESparseResourceAllocation snapShot =
getAvailableResourceOverTime(user, oldId, rStart, rEnd, 0);
// time-align on start
snapShot.shift(-(i * period));
// pick the minimum amount of resources in each time interval
minOverLCM =
RLESparseResourceAllocation.merge(resCalc, getTotalCapacity(),
minOverLCM, snapShot, RLEOperator.min, start, end);
}
return minOverLCM;
// add back in old reservation used resources if any
ReservationAllocation old = reservationTable.get(oldId);
if (old != null) {
netAvailable =
RLESparseResourceAllocation.merge(resCalc,
Resources.clone(totalCapacity), netAvailable,
old.getResourcesOverTime(), RLEOperator.add, start, end);
}
// lower it if this is needed by the sharing policy
netAvailable =
getSharingPolicy().availableResources(netAvailable, this, user,
oldId, start, end);
return netAvailable;
} finally {
readLock.unlock();
}
@ -637,7 +808,7 @@ public Resource getMaximumAllocation() {
public String toCumulativeString() {
readLock.lock();
try {
return rleSparseVector.toString();
return rleSparseVector.toString() + "\n" + periodicRle.toString();
} finally {
readLock.unlock();
}
@ -689,11 +860,18 @@ public Set<ReservationAllocation> getReservationByUserAtTime(String user,
}
@Override
public RLESparseResourceAllocation getCumulativeLoadOverTime(
long start, long end) {
public RLESparseResourceAllocation getCumulativeLoadOverTime(long start,
long end) throws PlanningException {
readLock.lock();
try {
return rleSparseVector.getRangeOverlapping(start, end);
RLESparseResourceAllocation ret =
rleSparseVector.getRangeOverlapping(start, end);
ret = RLESparseResourceAllocation.merge(resCalc, totalCapacity, ret,
periodicRle.getRangeOverlapping(start, end), RLEOperator.add, start,
end);
return ret;
} finally {
readLock.unlock();
}

View File

@ -42,6 +42,7 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
private final Map<ReservationInterval, Resource> allocationRequests;
private boolean hasGang = false;
private long acceptedAt = -1;
private long periodicity = 0;
private RLESparseResourceAllocation resourcesOverTime;
@ -67,9 +68,16 @@ public InMemoryReservationAllocation(ReservationId reservationID,
this.allocationRequests = allocations;
this.planName = planName;
this.hasGang = hasGang;
resourcesOverTime = new RLESparseResourceAllocation(calculator);
for (Map.Entry<ReservationInterval, Resource> r : allocations
.entrySet()) {
if (contract != null && contract.getRecurrenceExpression() != null) {
this.periodicity = Long.parseLong(contract.getRecurrenceExpression());
}
if (periodicity > 0) {
resourcesOverTime =
new PeriodicRLESparseResourceAllocation(calculator, periodicity);
} else {
resourcesOverTime = new RLESparseResourceAllocation(calculator);
}
for (Map.Entry<ReservationInterval, Resource> r : allocations.entrySet()) {
resourcesOverTime.addInterval(r.getKey(), r.getValue());
}
}
@ -133,17 +141,33 @@ public Resource getResourcesAtTime(long tick) {
}
@Override
public RLESparseResourceAllocation getResourcesOverTime(){
public RLESparseResourceAllocation getResourcesOverTime() {
return resourcesOverTime;
}
@Override
public RLESparseResourceAllocation getResourcesOverTime(long start,
long end) {
return resourcesOverTime.getRangeOverlapping(start, end);
}
@Override
public long getPeriodicity() {
return periodicity;
}
@Override
public void setPeriodicity(long period) {
periodicity = period;
}
@Override
public String toString() {
StringBuilder sBuf = new StringBuilder();
sBuf.append(getReservationId()).append(" user:").append(getUser())
.append(" startTime: ").append(getStartTime()).append(" endTime: ")
.append(getEndTime()).append(" alloc:\n[")
.append(resourcesOverTime.toString()).append("] ");
.append(getEndTime()).append(" Periodiciy: ").append(periodicity)
.append(" alloc:\n[").append(resourcesOverTime.toString()).append("] ");
return sBuf.toString();
}

View File

@ -40,7 +40,7 @@ public void validate(Plan plan, ReservationAllocation reservation)
RLESparseResourceAllocation available = plan.getAvailableResourceOverTime(
reservation.getUser(), reservation.getReservationId(),
reservation.getStartTime(), reservation.getEndTime());
reservation.getStartTime(), reservation.getEndTime(), 0);
// test the reservation does not exceed what is available
try {

View File

@ -18,47 +18,94 @@
package org.apache.hadoop.yarn.server.resourcemanager.reservation;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.util.resource.Resources;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
/**
* This data structure stores a periodic RLESparseResourceAllocation.
* This data structure stores a periodic {@link RLESparseResourceAllocation}.
* Default period is 1 day (86400000ms).
*/
public class PeriodicRLESparseResourceAllocation extends
RLESparseResourceAllocation {
public class PeriodicRLESparseResourceAllocation
extends RLESparseResourceAllocation {
// Log
private static final Logger LOG = LoggerFactory
.getLogger(PeriodicRLESparseResourceAllocation.class);
private static final Logger LOG =
LoggerFactory.getLogger(PeriodicRLESparseResourceAllocation.class);
private long timePeriod;
/**
* Constructor.
*
* @param rleVector {@link RLESparseResourceAllocation} with the run-length
encoded data.
* @param resourceCalculator {@link ResourceCalculator} the resource
* calculator to use.
* @param timePeriod Time period in milliseconds.
*/
public PeriodicRLESparseResourceAllocation(
RLESparseResourceAllocation rleVector, Long timePeriod) {
super(rleVector.getCumulative(), rleVector.getResourceCalculator());
ResourceCalculator resourceCalculator, Long timePeriod) {
super(resourceCalculator);
this.timePeriod = timePeriod;
}
/**
* Constructor. Default time period set to 1 day.
*
* @param rleVector {@link RLESparseResourceAllocation} with the run-length
encoded data.
* @param resourceCalculator {@link ResourceCalculator} the resource
* calculator to use..
*/
public PeriodicRLESparseResourceAllocation(
RLESparseResourceAllocation rleVector) {
this(rleVector, 86400000L);
ResourceCalculator resourceCalculator) {
this(resourceCalculator,
YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
}
/**
* Constructor.
*
* @param rleVector {@link RLESparseResourceAllocation} with the run-length
* encoded data.
* @param timePeriod Time period in milliseconds.
*/
@VisibleForTesting
public PeriodicRLESparseResourceAllocation(
RLESparseResourceAllocation rleVector, Long timePeriod) {
super(rleVector.getCumulative(), rleVector.getResourceCalculator());
this.timePeriod = timePeriod;
// make sure the PeriodicRLE is zero-based, and handles wrap-around
long delta = (getEarliestStartTime() % timePeriod - getEarliestStartTime());
shift(delta);
List<Long> toRemove = new ArrayList<>();
Map<Long, Resource> toAdd = new TreeMap<>();
for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
if (entry.getKey() > timePeriod) {
toRemove.add(entry.getKey());
if (entry.getValue() != null) {
toAdd.put(timePeriod, entry.getValue());
long prev = entry.getKey() % timePeriod;
toAdd.put(prev, this.getCapacityAtTime(prev));
toAdd.put(0L, entry.getValue());
}
}
}
for (Long l : toRemove) {
cumulativeCapacity.remove(l);
}
cumulativeCapacity.putAll(toAdd);
}
/**
@ -78,24 +125,25 @@ public Resource getCapacityAtTime(long tick) {
* The interval may include 0, but the end time must be strictly less than
* timePeriod.
*
* @param interval {@link ReservationInterval} to which the specified
* resource is to be added.
* @param interval {@link ReservationInterval} to which the specified resource
* is to be added.
* @param resource {@link Resource} to be added to the interval specified.
* @return true if addition is successful, false otherwise
*/
public boolean addInterval(ReservationInterval interval,
Resource resource) {
public boolean addInterval(ReservationInterval interval, Resource resource) {
long startTime = interval.getStartTime();
long endTime = interval.getEndTime();
if (startTime >= 0 && endTime > startTime && endTime <= timePeriod) {
return super.addInterval(interval, resource);
} else {
LOG.info("Cannot set capacity beyond end time: " + timePeriod);
LOG.info("Cannot set capacity beyond end time: " + timePeriod + " was ("
+ interval.toString() + ")");
return false;
}
}
/**
/**
* Removes a resource for the specified interval.
*
* @param interval the {@link ReservationInterval} for which the resource is
@ -103,14 +151,15 @@ public boolean addInterval(ReservationInterval interval,
* @param resource the {@link Resource} to be removed.
* @return true if removal is successful, false otherwise
*/
public boolean removeInterval(
ReservationInterval interval, Resource resource) {
public boolean removeInterval(ReservationInterval interval,
Resource resource) {
long startTime = interval.getStartTime();
long endTime = interval.getEndTime();
// If the resource to be subtracted is less than the minimum resource in
// the range, abort removal to avoid negative capacity.
if (!Resources.fitsIn(
resource, super.getMinimumCapacityInInterval(interval))) {
// TODO revesit decrementing endTime
if (!Resources.fitsIn(resource, getMinimumCapacityInInterval(
new ReservationInterval(startTime, endTime - 1)))) {
LOG.info("Request to remove more resources than what is available");
return false;
}
@ -125,17 +174,16 @@ public boolean removeInterval(
/**
* Get maximum capacity at periodic offsets from the specified time.
*
* @param tick UTC time base from which offsets are specified for finding
* the maximum capacity.
* @param period periodic offset at which capacities are evaluted.
* @param tick UTC time base from which offsets are specified for finding the
* maximum capacity.
* @param period periodic offset at which capacities are evaluated.
* @return the maximum {@link Resource} across the specified time instants.
* @return true if removal is successful, false otherwise
*/
public Resource getMaximumPeriodicCapacity(long tick, long period) {
Resource maxResource;
if (period < timePeriod) {
maxResource =
super.getMaximumPeriodicCapacity(tick % timePeriod, period);
maxResource = super.getMaximumPeriodicCapacity(tick % timePeriod, period);
} else {
// if period is greater than the length of PeriodicRLESparseAllocation,
// only a single value exists in this interval.
@ -164,4 +212,30 @@ public String toString() {
return ret.toString();
}
@Override
public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
NavigableMap<Long, Resource> unrolledMap = new TreeMap<>();
readLock.lock();
try {
long relativeStart = (start >= 0) ? start % timePeriod : 0;
NavigableMap<Long, Resource> cumulativeMap = this.getCumulative();
Long previous = cumulativeMap.floorKey(relativeStart);
previous = (previous != null) ? previous : 0;
for (long i = 0; i <= (end - start) / timePeriod; i++) {
for (Map.Entry<Long, Resource> e : cumulativeMap.entrySet()) {
long curKey = e.getKey() + (i * timePeriod);
if (curKey >= previous && (start + curKey - relativeStart) <= end) {
unrolledMap.put(curKey, e.getValue());
}
}
}
RLESparseResourceAllocation rle =
new RLESparseResourceAllocation(unrolledMap, getResourceCalculator());
rle.shift(start - relativeStart);
return rle;
} finally {
readLock.unlock();
}
}
}

View File

@ -28,54 +28,58 @@
public interface PlanEdit extends PlanContext, PlanView {
/**
* Add a new {@link ReservationAllocation} to the plan
* Add a new {@link ReservationAllocation} to the plan.
*
* @param reservation the {@link ReservationAllocation} to be added to the
* plan
* @param isRecovering flag to indicate if reservation is being added as part
* of failover or not
* @return true if addition is successful, false otherwise
* @throws PlanningException if addition is unsuccessful
*/
public boolean addReservation(ReservationAllocation reservation,
boolean addReservation(ReservationAllocation reservation,
boolean isRecovering) throws PlanningException;
/**
* Updates an existing {@link ReservationAllocation} in the plan. This is
* required for re-negotiation
* required for re-negotiation.
*
* @param reservation the {@link ReservationAllocation} to be updated the plan
* @return true if update is successful, false otherwise
* @throws PlanningException if update is unsuccessful
*/
public boolean updateReservation(ReservationAllocation reservation)
boolean updateReservation(ReservationAllocation reservation)
throws PlanningException;
/**
* Delete an existing {@link ReservationAllocation} from the plan identified
* uniquely by its {@link ReservationId}. This will generally be used for
* garbage collection
* garbage collection.
*
* @param reservationID the {@link ReservationAllocation} to be deleted from
* the plan identified uniquely by its {@link ReservationId}
* @return true if delete is successful, false otherwise
* @throws PlanningException if deletion is unsuccessful
*/
public boolean deleteReservation(ReservationId reservationID)
boolean deleteReservation(ReservationId reservationID)
throws PlanningException;
/**
* Method invoked to garbage collect old reservations. It cleans up expired
* reservations that have fallen out of the sliding archival window
* reservations that have fallen out of the sliding archival window.
*
* @param tick the current time from which the archival window is computed
* @throws PlanningException if archival is unsuccessful
*/
public void archiveCompletedReservations(long tick) throws PlanningException;
void archiveCompletedReservations(long tick) throws PlanningException;
/**
* Sets the overall capacity in terms of {@link Resource} assigned to this
* plan
* plan.
*
* @param capacity the overall capacity in terms of {@link Resource} assigned
* to this plan
*/
public void setTotalCapacity(Resource capacity);
void setTotalCapacity(Resource capacity);
}

View File

@ -17,50 +17,50 @@
*/
package org.apache.hadoop.yarn.server.resourcemanager.reservation;
import java.util.Set;
import org.apache.hadoop.yarn.api.records.ReservationId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
import java.util.Set;
/**
* This interface provides a read-only view on the allocations made in this
* plan. This methods are used for example by {@code ReservationAgent}s to
* determine the free resources in a certain point in time, and by
* PlanFollowerPolicy to publish this plan to the scheduler.
*/
public interface PlanView extends PlanContext {
interface PlanView extends PlanContext {
/**
* Return a set of {@link ReservationAllocation} identified by the user who
* made the reservation.
*
* @param reservationID the unqiue id to identify the
* {@link ReservationAllocation}
* {@link ReservationAllocation}
* @param interval the time interval used to retrieve the reservation
* allocations from. Only reservations with start time no
* greater than the interval end time, and end time no less
* than the interval start time will be selected.
* allocations from. Only reservations with start time no greater
* than the interval end time, and end time no less than the interval
* start time will be selected.
* @param user the user to retrieve the reservation allocation from.
* @return a set of {@link ReservationAllocation} identified by the user who
* made the reservation
* made the reservation
*/
Set<ReservationAllocation> getReservations(ReservationId
reservationID, ReservationInterval interval, String user);
Set<ReservationAllocation> getReservations(ReservationId reservationID,
ReservationInterval interval, String user);
/**
* Return a set of {@link ReservationAllocation} identified by any user.
*
* @param reservationID the unqiue id to identify the
* {@link ReservationAllocation}
* {@link ReservationAllocation}
* @param interval the time interval used to retrieve the reservation
* allocations from. Only reservations with start time no
* greater than the interval end time, and end time no less
* than the interval start time will be selected.
* allocations from. Only reservations with start time no greater
* than the interval end time, and end time no less than the interval
* start time will be selected.
* @return a set of {@link ReservationAllocation} identified by any user
*/
Set<ReservationAllocation> getReservations(ReservationId reservationID,
ReservationInterval interval);
ReservationInterval interval);
/**
* Return a {@link ReservationAllocation} identified by its
@ -70,7 +70,7 @@ Set<ReservationAllocation> getReservations(ReservationId reservationID,
* {@link ReservationAllocation}
* @return {@link ReservationAllocation} identified by the specified id
*/
public ReservationAllocation getReservationById(ReservationId reservationID);
ReservationAllocation getReservationById(ReservationId reservationID);
/**
* Return a set of {@link ReservationAllocation} that belongs to a certain
@ -78,11 +78,10 @@ Set<ReservationAllocation> getReservations(ReservationId reservationID,
*
* @param user the user being considered
* @param t the instant in time being considered
* @return set of active {@link ReservationAllocation}s for this
* user at this time
* @return set of active {@link ReservationAllocation}s for this user at this
* time
*/
public Set<ReservationAllocation> getReservationByUserAtTime(String user,
long t);
Set<ReservationAllocation> getReservationByUserAtTime(String user, long t);
/**
* Gets all the active reservations at the specified point of time
@ -91,14 +90,14 @@ public Set<ReservationAllocation> getReservationByUserAtTime(String user,
* requested
* @return set of active reservations at the specified time
*/
public Set<ReservationAllocation> getReservationsAtTime(long tick);
Set<ReservationAllocation> getReservationsAtTime(long tick);
/**
* Gets all the reservations in the plan
*
* @return set of all reservations handled by this Plan
*/
public Set<ReservationAllocation> getAllReservations();
Set<ReservationAllocation> getAllReservations();
/**
* Returns the total {@link Resource} reserved for all users at the specified
@ -126,61 +125,68 @@ public Set<ReservationAllocation> getReservationByUserAtTime(String user,
*
* @return the time (UTC in ms) at which the first reservation starts
*/
public long getEarliestStartTime();
long getEarliestStartTime();
/**
* Returns the time (UTC in ms) at which the last reservation terminates
*
* @return the time (UTC in ms) at which the last reservation terminates
*/
public long getLastEndTime();
long getLastEndTime();
/**
* This method returns the amount of resources available to a given user
* (optionally if removing a certain reservation) over the start-end time
* range.
* range. If the request is periodic (period is non-zero) we return the
* minimum amount of resources available to periodic reservations (in all
* "period" windows within the system maxPeriod / LCM).
*
* @param user
* @param oldId
* @param start
* @param end
* @param user the user being considered
* @param oldId the identifier of the existing reservation
* @param start start of the time interval.
* @param end end of the time interval.
* @param period the ms periodicty for this request (loop and pick min till
* maxPeriodicity)
* @return a view of the plan as it is available to this user
* @throws PlanningException
* @throws PlanningException if operation is unsuccessful
*/
public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
ReservationId oldId, long start, long end) throws PlanningException;
RLESparseResourceAllocation getAvailableResourceOverTime(String user,
ReservationId oldId, long start, long end, long period)
throws PlanningException;
/**
* This method returns a RLE encoded view of the user reservation count
* utilization between start and end time.
*
* @param user
* @param start
* @param end
* @param user the user being considered
* @param start start of the time interval.
* @param end end of the time interval.
* @return RLE encoded view of reservation used over time
*/
public RLESparseResourceAllocation getReservationCountForUserOverTime(
String user, long start, long end);
RLESparseResourceAllocation getReservationCountForUserOverTime(String user,
long start, long end);
/**
* This method returns a RLE encoded view of the user reservation utilization
* between start and end time.
*
* @param user
* @param start
* @param end
* @param user the user being considered
* @param start start of the time interval.
* @param end end of the time interval.
* @return RLE encoded view of resources used over time
*/
public RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
long start, long end);
/**
* Get the cumulative load over a time interval.
*
* @param start Start of the time interval.
* @param end End of the time interval.
* @param start start of the time interval.
* @param end end of the time interval.
* @return RLE sparse allocation.
* @throws PlanningException if operation is unsuccessful
*/
RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end);
RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end)
throws PlanningException;
}

View File

@ -18,8 +18,6 @@
package org.apache.hadoop.yarn.server.resourcemanager.reservation;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Iterator;
import java.util.Map;
import java.util.Map.Entry;
@ -33,8 +31,6 @@
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.gson.stream.JsonWriter;
/**
* This is a run length encoded sparse data structure that maintains resource
* allocations over time.
@ -44,12 +40,14 @@ public class RLESparseResourceAllocation {
private static final int THRESHOLD = 100;
private static final Resource ZERO_RESOURCE = Resources.none();
private NavigableMap<Long, Resource> cumulativeCapacity =
@SuppressWarnings("checkstyle:visibilitymodifier")
protected NavigableMap<Long, Resource> cumulativeCapacity =
new TreeMap<Long, Resource>();
private final ReentrantReadWriteLock readWriteLock =
new ReentrantReadWriteLock();
private final Lock readLock = readWriteLock.readLock();
@SuppressWarnings("checkstyle:visibilitymodifier")
protected final Lock readLock = readWriteLock.readLock();
private final Lock writeLock = readWriteLock.writeLock();
private final ResourceCalculator resourceCalculator;
@ -235,34 +233,6 @@ public String toString() {
}
}
/**
* Returns the JSON string representation of the current resources allocated
* over time.
*
* @return the JSON string representation of the current resources allocated
* over time
*/
public String toMemJSONString() {
StringWriter json = new StringWriter();
JsonWriter jsonWriter = new JsonWriter(json);
readLock.lock();
try {
jsonWriter.beginObject();
// jsonWriter.name("timestamp").value("resource");
for (Map.Entry<Long, Resource> r : cumulativeCapacity.entrySet()) {
jsonWriter.name(r.getKey().toString()).value(r.getValue().toString());
}
jsonWriter.endObject();
jsonWriter.close();
return json.toString();
} catch (IOException e) {
// This should not happen
return "";
} finally {
readLock.unlock();
}
}
/**
* Returns the representation of the current resources allocated over time as
* an interval map (in the defined non-null range).
@ -437,8 +407,8 @@ private static NavigableMap<Long, Resource> negate(RLEOperator operator,
Resource val = Resources.negate(e.getValue());
// test for negative value and throws
if (operator == RLEOperator.subtractTestNonNegative
&& (Resources.fitsIn(val, ZERO_RESOURCE) &&
!Resources.equals(val, ZERO_RESOURCE))) {
&& (Resources.fitsIn(val, ZERO_RESOURCE)
&& !Resources.equals(val, ZERO_RESOURCE))) {
throw new PlanningException(
"RLESparseResourceAllocation: merge failed as the "
+ "resulting RLESparseResourceAllocation would be negative");
@ -504,22 +474,29 @@ private static Resource combineValue(RLEOperator op,
}
/**
* Get a {@link RLESparseResourceAllocation} view of the {@link Resource}
* allocations between the specified start and end times.
*
* @param start the time from which the {@link Resource} allocations are
* required
* @param end the time upto which the {@link Resource} allocations are
* required
* @return the overlapping allocations
*/
public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
readLock.lock();
try {
NavigableMap<Long, Resource> a = this.getCumulative();
if (a != null && !a.isEmpty()) {
// include the portion of previous entry that overlaps start
if (start > a.firstKey()) {
long previous = a.floorKey(start);
a = a.tailMap(previous, true);
}
if (end < a.lastKey()) {
a = a.headMap(end, true);
}
}
RLESparseResourceAllocation ret =
new RLESparseResourceAllocation(a, resourceCalculator);
@ -527,7 +504,33 @@ public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
} finally {
readLock.unlock();
}
}
/**
* This method shifts all the timestamp of the {@link Resource} entries by the
* specified "delta".
*
* @param delta the time by which to shift the {@link Resource} allocations
*/
public void shift(long delta) {
writeLock.lock();
try {
TreeMap<Long, Resource> newCum = new TreeMap<>();
long start;
for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
if (delta > 0) {
start = (entry.getKey() == Long.MAX_VALUE) ? Long.MAX_VALUE
: entry.getKey() + delta;
} else {
start = (entry.getKey() == Long.MIN_VALUE) ? Long.MIN_VALUE
: entry.getKey() + delta;
}
newCum.put(start, entry.getValue());
}
cumulativeCapacity = newCum;
} finally {
writeLock.unlock();
}
}
/**
@ -541,8 +544,8 @@ public enum RLEOperator {
/**
* Get the maximum capacity across specified time instances. The search-space
* is specified using the starting value, tick, and the periodic interval for
* search. Maximum resource allocation across tick, tick + period,
* tick + 2 * period,..., tick + n * period .. is returned.
* search. Maximum resource allocation across tick, tick + period, tick + 2 *
* period,..., tick + n * period .. is returned.
*
* @param tick the starting time instance
* @param period interval at which capacity is evaluated
@ -550,14 +553,19 @@ public enum RLEOperator {
*/
public Resource getMaximumPeriodicCapacity(long tick, long period) {
Resource maxCapacity = ZERO_RESOURCE;
if (!cumulativeCapacity.isEmpty()) {
Long lastKey = cumulativeCapacity.lastKey();
for (long t = tick; t <= lastKey; t = t + period) {
maxCapacity = Resources.componentwiseMax(maxCapacity,
cumulativeCapacity.floorEntry(t).getValue());
readLock.lock();
try {
if (!cumulativeCapacity.isEmpty()) {
Long lastKey = cumulativeCapacity.lastKey();
for (long t = tick; t <= lastKey; t = t + period) {
maxCapacity = Resources.componentwiseMax(maxCapacity,
cumulativeCapacity.floorEntry(t).getValue());
}
}
return maxCapacity;
} finally {
readLock.unlock();
}
return maxCapacity;
}
/**
@ -567,17 +575,17 @@ public Resource getMaximumPeriodicCapacity(long tick, long period) {
* @return minimum resource allocation
*/
public Resource getMinimumCapacityInInterval(ReservationInterval interval) {
Resource minCapacity = Resource.newInstance(
Integer.MAX_VALUE, Integer.MAX_VALUE);
Resource minCapacity =
Resource.newInstance(Integer.MAX_VALUE, Integer.MAX_VALUE);
long start = interval.getStartTime();
long end = interval.getEndTime();
NavigableMap<Long, Resource> capacityRange =
this.getRangeOverlapping(start, end).getCumulative();
getRangeOverlapping(start, end).getCumulative();
if (!capacityRange.isEmpty()) {
for (Map.Entry<Long, Resource> entry : capacityRange.entrySet()) {
if (entry.getValue() != null) {
minCapacity = Resources.componentwiseMin(minCapacity,
entry.getValue());
minCapacity =
Resources.componentwiseMin(minCapacity, entry.getValue());
}
}
}

View File

@ -24,14 +24,16 @@
import org.apache.hadoop.yarn.api.records.ReservationId;
import org.apache.hadoop.yarn.api.records.Resource;
import com.google.common.annotations.VisibleForTesting;
/**
* A ReservationAllocation represents a concrete allocation of resources over
* time that satisfy a certain {@link ReservationDefinition}. This is used
* internally by a {@link Plan} to store information about how each of the
* accepted {@link ReservationDefinition} have been allocated.
*/
public interface ReservationAllocation extends
Comparable<ReservationAllocation> {
public interface ReservationAllocation
extends Comparable<ReservationAllocation> {
/**
* Returns the unique identifier {@link ReservationId} that represents the
@ -40,28 +42,28 @@ public interface ReservationAllocation extends
* @return reservationId the unique identifier {@link ReservationId} that
* represents the reservation
*/
public ReservationId getReservationId();
ReservationId getReservationId();
/**
* Returns the original {@link ReservationDefinition} submitted by the client
*
* @return the {@link ReservationDefinition} submitted by the client
*/
public ReservationDefinition getReservationDefinition();
ReservationDefinition getReservationDefinition();
/**
* Returns the time at which the reservation is activated.
*
* @return the time at which the reservation is activated
*/
public long getStartTime();
long getStartTime();
/**
* Returns the time at which the reservation terminates.
*
* @return the time at which the reservation terminates
*/
public long getEndTime();
long getEndTime();
/**
* Returns the map of resources requested against the time interval for which
@ -70,28 +72,28 @@ public interface ReservationAllocation extends
* @return the allocationRequests the map of resources requested against the
* time interval for which they were
*/
public Map<ReservationInterval, Resource> getAllocationRequests();
Map<ReservationInterval, Resource> getAllocationRequests();
/**
* Return a string identifying the plan to which the reservation belongs
*
* @return the plan to which the reservation belongs
*/
public String getPlanName();
String getPlanName();
/**
* Returns the user who requested the reservation
*
* @return the user who requested the reservation
*/
public String getUser();
String getUser();
/**
* Returns whether the reservation has gang semantics or not
*
* @return true if there is a gang request, false otherwise
*/
public boolean containsGangs();
boolean containsGangs();
/**
* Sets the time at which the reservation was accepted by the system
@ -99,14 +101,14 @@ public interface ReservationAllocation extends
* @param acceptedAt the time at which the reservation was accepted by the
* system
*/
public void setAcceptanceTimestamp(long acceptedAt);
void setAcceptanceTimestamp(long acceptedAt);
/**
* Returns the time at which the reservation was accepted by the system
*
* @return the time at which the reservation was accepted by the system
*/
public long getAcceptanceTime();
long getAcceptanceTime();
/**
* Returns the capacity represented by cumulative resources reserved by the
@ -116,12 +118,42 @@ public interface ReservationAllocation extends
* requested
* @return the resources reserved at the specified time
*/
public Resource getResourcesAtTime(long tick);
Resource getResourcesAtTime(long tick);
/**
* Return a RLE representation of used resources.
*
* @return a RLE encoding of resources allocated over time.
*/
public RLESparseResourceAllocation getResourcesOverTime();
RLESparseResourceAllocation getResourcesOverTime();
/**
* Return a RLE representation of used resources.
*
* @param start start of the time interval.
* @param end end of the time interval.
* @return a RLE encoding of resources allocated over time.
*/
RLESparseResourceAllocation getResourcesOverTime(long start, long end);
/**
* Get the periodicity of this reservation representing the time period of the
* periodic job. Period is represented in milliseconds for periodic jobs.
* Period is 0 for non-periodic jobs.
*
* @return periodicity of this reservation
*/
long getPeriodicity();
/**
* Set the periodicity of this reservation representing the time period of the
* periodic job. Period is represented in milliseconds for periodic jobs.
* Period is 0 for non-periodic jobs.
*
* @param period periodicity of this reservation
*/
@VisibleForTesting
void setPeriodicity(long period);
}

View File

@ -44,6 +44,8 @@ public class ReservationInputValidator {
/**
* Utility class to validate reservation requests.
*
* @param clock the {@link Clock} to use
*/
public ReservationInputValidator(Clock clock) {
this.clock = clock;
@ -53,22 +55,21 @@ private Plan validateReservation(ReservationSystem reservationSystem,
ReservationId reservationId, String auditConstant) throws YarnException {
// check if the reservation id is valid
if (reservationId == null) {
String message =
"Missing reservation id."
+ " Please try again by specifying a reservation id.";
String message = "Missing reservation id."
+ " Please try again by specifying a reservation id.";
RMAuditLogger.logFailure("UNKNOWN", auditConstant,
"validate reservation input", "ClientRMService", message);
throw RPCUtil.getRemoteException(message);
}
String queue = reservationSystem.getQueueForReservation(reservationId);
String nullQueueErrorMessage =
"The specified reservation with ID: " + reservationId
+ " is unknown. Please try again with a valid reservation.";
"The specified reservation with ID: " + reservationId
+ " is unknown. Please try again with a valid reservation.";
String nullPlanErrorMessage = "The specified reservation: " + reservationId
+ " is not associated with any valid plan."
+ " Please try again with a valid reservation.";
+ " is not associated with any valid plan."
+ " Please try again with a valid reservation.";
return getPlanFromQueue(reservationSystem, queue, auditConstant,
nullQueueErrorMessage, nullPlanErrorMessage);
nullQueueErrorMessage, nullPlanErrorMessage);
}
private void validateReservationDefinition(ReservationId reservationId,
@ -77,17 +78,15 @@ private void validateReservationDefinition(ReservationId reservationId,
String message = "";
// check if deadline is in the past
if (contract == null) {
message =
"Missing reservation definition."
+ " Please try again by specifying a reservation definition.";
message = "Missing reservation definition."
+ " Please try again by specifying a reservation definition.";
RMAuditLogger.logFailure("UNKNOWN", auditConstant,
"validate reservation input definition", "ClientRMService", message);
throw RPCUtil.getRemoteException(message);
}
if (contract.getDeadline() <= clock.getTime()) {
message =
"The specified deadline: " + contract.getDeadline()
+ " is the past. Please try again with deadline in the future.";
message = "The specified deadline: " + contract.getDeadline()
+ " is the past. Please try again with deadline in the future.";
RMAuditLogger.logFailure("UNKNOWN", auditConstant,
"validate reservation input definition", "ClientRMService", message);
throw RPCUtil.getRemoteException(message);
@ -95,18 +94,16 @@ private void validateReservationDefinition(ReservationId reservationId,
// Check if at least one RR has been specified
ReservationRequests resReqs = contract.getReservationRequests();
if (resReqs == null) {
message =
"No resources have been specified to reserve."
+ "Please try again by specifying the resources to reserve.";
message = "No resources have been specified to reserve."
+ "Please try again by specifying the resources to reserve.";
RMAuditLogger.logFailure("UNKNOWN", auditConstant,
"validate reservation input definition", "ClientRMService", message);
throw RPCUtil.getRemoteException(message);
}
List<ReservationRequest> resReq = resReqs.getReservationResources();
if (resReq == null || resReq.isEmpty()) {
message =
"No resources have been specified to reserve."
+ " Please try again by specifying the resources to reserve.";
message = "No resources have been specified to reserve."
+ " Please try again by specifying the resources to reserve.";
RMAuditLogger.logFailure("UNKNOWN", auditConstant,
"validate reservation input definition", "ClientRMService", message);
throw RPCUtil.getRemoteException(message);
@ -123,22 +120,18 @@ private void validateReservationDefinition(ReservationId reservationId,
} else {
minDuration += rr.getDuration();
}
maxGangSize =
Resources.max(plan.getResourceCalculator(), plan.getTotalCapacity(),
maxGangSize,
Resources.multiply(rr.getCapability(), rr.getConcurrency()));
maxGangSize = Resources.max(plan.getResourceCalculator(),
plan.getTotalCapacity(), maxGangSize,
Resources.multiply(rr.getCapability(), rr.getConcurrency()));
}
// verify the allocation is possible (skip for ANY)
long duration = contract.getDeadline() - contract.getArrival();
if (duration < minDuration
&& type != ReservationRequestInterpreter.R_ANY) {
message =
"The time difference ("
+ (duration)
+ ") between arrival (" + contract.getArrival() + ") "
+ "and deadline (" + contract.getDeadline() + ") must "
+ " be greater or equal to the minimum resource duration ("
+ minDuration + ")";
if (duration < minDuration && type != ReservationRequestInterpreter.R_ANY) {
message = "The time difference (" + (duration) + ") between arrival ("
+ contract.getArrival() + ") " + "and deadline ("
+ contract.getDeadline() + ") must "
+ " be greater or equal to the minimum resource duration ("
+ minDuration + ")";
RMAuditLogger.logFailure("UNKNOWN", auditConstant,
"validate reservation input definition", "ClientRMService", message);
throw RPCUtil.getRemoteException(message);
@ -148,10 +141,9 @@ private void validateReservationDefinition(ReservationId reservationId,
if (Resources.greaterThan(plan.getResourceCalculator(),
plan.getTotalCapacity(), maxGangSize, plan.getTotalCapacity())
&& type != ReservationRequestInterpreter.R_ANY) {
message =
"The size of the largest gang in the reservation definition ("
+ maxGangSize + ") exceed the capacity available ("
+ plan.getTotalCapacity() + " )";
message = "The size of the largest gang in the reservation definition ("
+ maxGangSize + ") exceed the capacity available ("
+ plan.getTotalCapacity() + " )";
RMAuditLogger.logFailure("UNKNOWN", auditConstant,
"validate reservation input definition", "ClientRMService", message);
throw RPCUtil.getRemoteException(message);
@ -179,32 +171,32 @@ private void validateReservationDefinition(ReservationId reservationId,
}
}
private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
queue, String auditConstant) throws YarnException {
private Plan getPlanFromQueue(ReservationSystem reservationSystem,
String queue, String auditConstant) throws YarnException {
String nullQueueErrorMessage = "The queue is not specified."
+ " Please try again with a valid reservable queue.";
+ " Please try again with a valid reservable queue.";
String nullPlanErrorMessage = "The specified queue: " + queue
+ " is not managed by reservation system."
+ " Please try again with a valid reservable queue.";
+ " is not managed by reservation system."
+ " Please try again with a valid reservable queue.";
return getPlanFromQueue(reservationSystem, queue, auditConstant,
nullQueueErrorMessage, nullPlanErrorMessage);
nullQueueErrorMessage, nullPlanErrorMessage);
}
private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
queue, String auditConstant, String nullQueueErrorMessage,
String nullPlanErrorMessage) throws YarnException {
private Plan getPlanFromQueue(ReservationSystem reservationSystem,
String queue, String auditConstant, String nullQueueErrorMessage,
String nullPlanErrorMessage) throws YarnException {
if (queue == null || queue.isEmpty()) {
RMAuditLogger.logFailure("UNKNOWN", auditConstant,
"validate reservation input", "ClientRMService",
nullQueueErrorMessage);
"validate reservation input", "ClientRMService",
nullQueueErrorMessage);
throw RPCUtil.getRemoteException(nullQueueErrorMessage);
}
// check if the associated plan is valid
Plan plan = reservationSystem.getPlan(queue);
if (plan == null) {
RMAuditLogger.logFailure("UNKNOWN", auditConstant,
"validate reservation input", "ClientRMService",
nullPlanErrorMessage);
"validate reservation input", "ClientRMService",
nullPlanErrorMessage);
throw RPCUtil.getRemoteException(nullPlanErrorMessage);
}
return plan;
@ -222,22 +214,21 @@ private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
* @param reservationId the {@link ReservationId} associated with the current
* request
* @return the {@link Plan} to submit the request to
* @throws YarnException
* @throws YarnException if validation fails
*/
public Plan validateReservationSubmissionRequest(
ReservationSystem reservationSystem,
ReservationSubmissionRequest request, ReservationId reservationId)
throws YarnException {
ReservationSystem reservationSystem, ReservationSubmissionRequest request,
ReservationId reservationId) throws YarnException {
String message;
if (reservationId == null) {
message = "Reservation id cannot be null. Please try again " +
"specifying a valid reservation id by creating a new reservation id.";
message = "Reservation id cannot be null. Please try again specifying "
+ " a valid reservation id by creating a new reservation id.";
throw RPCUtil.getRemoteException(message);
}
// Check if it is a managed queue
String queue = request.getQueue();
Plan plan = getPlanFromQueue(reservationSystem, queue,
AuditConstants.SUBMIT_RESERVATION_REQUEST);
AuditConstants.SUBMIT_RESERVATION_REQUEST);
validateReservationDefinition(reservationId,
request.getReservationDefinition(), plan,
@ -255,15 +246,14 @@ public Plan validateReservationSubmissionRequest(
* @param request the {@link ReservationUpdateRequest} defining the resources
* required over time for the request
* @return the {@link Plan} to submit the request to
* @throws YarnException
* @throws YarnException if validation fails
*/
public Plan validateReservationUpdateRequest(
ReservationSystem reservationSystem, ReservationUpdateRequest request)
throws YarnException {
ReservationId reservationId = request.getReservationId();
Plan plan =
validateReservation(reservationSystem, reservationId,
AuditConstants.UPDATE_RESERVATION_REQUEST);
Plan plan = validateReservation(reservationSystem, reservationId,
AuditConstants.UPDATE_RESERVATION_REQUEST);
validateReservationDefinition(reservationId,
request.getReservationDefinition(), plan,
AuditConstants.UPDATE_RESERVATION_REQUEST);
@ -278,28 +268,26 @@ public Plan validateReservationUpdateRequest(
*
* @param reservationSystem the {@link ReservationSystem} to validate against
* @param request the {@link ReservationListRequest} defining search
* parameters for reservations in the {@link ReservationSystem}
* that is being validated against.
* parameters for reservations in the {@link ReservationSystem} that
* is being validated against.
* @return the {@link Plan} to list reservations of.
* @throws YarnException
* @throws YarnException if validation fails
*/
public Plan validateReservationListRequest(
ReservationSystem reservationSystem,
ReservationListRequest request)
ReservationSystem reservationSystem, ReservationListRequest request)
throws YarnException {
String queue = request.getQueue();
if (request.getEndTime() < request.getStartTime()) {
String errorMessage = "The specified end time must be greater than " +
"the specified start time.";
String errorMessage = "The specified end time must be greater than "
+ "the specified start time.";
RMAuditLogger.logFailure("UNKNOWN",
AuditConstants.LIST_RESERVATION_REQUEST,
"validate list reservation input", "ClientRMService",
errorMessage);
AuditConstants.LIST_RESERVATION_REQUEST,
"validate list reservation input", "ClientRMService", errorMessage);
throw RPCUtil.getRemoteException(errorMessage);
}
// Check if it is a managed queue
return getPlanFromQueue(reservationSystem, queue,
AuditConstants.LIST_RESERVATION_REQUEST);
AuditConstants.LIST_RESERVATION_REQUEST);
}
/**
@ -312,7 +300,7 @@ public Plan validateReservationListRequest(
* @param request the {@link ReservationDeleteRequest} defining the resources
* required over time for the request
* @return the {@link Plan} to submit the request to
* @throws YarnException
* @throws YarnException if validation fails
*/
public Plan validateReservationDeleteRequest(
ReservationSystem reservationSystem, ReservationDeleteRequest request)

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.yarn.server.resourcemanager.reservation;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.conf.Configuration;
@ -29,8 +31,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.security.ReservationsACLsManager;
import java.util.Map;
/**
* This interface is the one implemented by any system that wants to support
* Reservations i.e. make {@code Resource} allocations in future. Implementors
@ -57,7 +57,7 @@ public interface ReservationSystem extends Recoverable {
*
* @param conf configuration
* @param rmContext current context of the {@code ResourceManager}
* @throws YarnException
* @throws YarnException if initialization of the configured plan fails
*/
void reinitialize(Configuration conf, RMContext rmContext)
throws YarnException;

View File

@ -38,7 +38,7 @@ public interface SharingPolicy {
* @param planQueuePath the name of the queue for this plan
* @param conf the system configuration
*/
public void init(String planQueuePath, ReservationSchedulerConfiguration conf);
void init(String planQueuePath, ReservationSchedulerConfiguration conf);
/**
* This method runs the policy validation logic, and return true/false on
@ -51,7 +51,7 @@ public interface SharingPolicy {
* @throws PlanningException if the policy is respected if we add this
* {@link ReservationAllocation} to the {@link Plan}
*/
public void validate(Plan plan, ReservationAllocation newAllocation)
void validate(Plan plan, ReservationAllocation newAllocation)
throws PlanningException;
/**
@ -68,9 +68,13 @@ public void validate(Plan plan, ReservationAllocation newAllocation)
* @param start the start time for the range we are querying
* @param end the end time for the range we are querying
* @param oldId (optional) the id of a reservation being updated
*
* @return the available resources expressed as a
* {@link RLESparseResourceAllocation}
*
* @throws PlanningException throws if the request is not valid
*/
public RLESparseResourceAllocation availableResources(
RLESparseResourceAllocation availableResources(
RLESparseResourceAllocation available, Plan plan, String user,
ReservationId oldId, long start, long end) throws PlanningException;
@ -82,7 +86,6 @@ public RLESparseResourceAllocation availableResources(
*
* @return validWindow the window of validity considered by the policy.
*/
public long getValidWindow();
long getValidWindow();
}

View File

@ -34,7 +34,7 @@ public interface Planner {
*
* @param plan the {@link Plan} to replan
* @param contracts the list of reservation requests
* @throws PlanningException
* @throws PlanningException if operation is unsuccessful
*/
public void plan(Plan plan, List<ReservationDefinition> contracts)
throws PlanningException;

View File

@ -50,7 +50,7 @@ public abstract class PlanningAlgorithm implements ReservationAgent {
* @return whether the allocateUser function was successful or not
*
* @throws PlanningException if the session cannot be fitted into the plan
* @throws ContractValidationException
* @throws ContractValidationException if validation fails
*/
protected boolean allocateUser(ReservationId reservationId, String user,
Plan plan, ReservationDefinition contract,

View File

@ -50,7 +50,7 @@ public interface StageAllocator {
*
* @return The computed allocation (or null if the stage could not be
* allocated)
* @throws PlanningException
* @throws PlanningException if operation is unsuccessful
*/
Map<ReservationInterval, Resource> computeStageAllocation(Plan plan,
RLESparseResourceAllocation planLoads,

View File

@ -69,7 +69,7 @@ public Map<ReservationInterval, Resource> computeStageAllocation(Plan plan,
RLESparseResourceAllocation netAvailable =
plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
stageDeadline);
stageDeadline, 0);
netAvailable =
RLESparseResourceAllocation.merge(plan.getResourceCalculator(),

View File

@ -83,9 +83,8 @@ public Map<ReservationInterval, Resource> computeStageAllocation(Plan plan,
int gangsToPlace = rr.getNumContainers() / rr.getConcurrency();
// get available resources from plan
RLESparseResourceAllocation netRLERes =
plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
stageDeadline);
RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
user, oldId, stageEarliestStart, stageDeadline, 0);
// remove plan modifications
netRLERes =

View File

@ -77,8 +77,8 @@ public Map<ReservationInterval, Resource> computeStageAllocation(Plan plan,
ResourceCalculator resCalc = plan.getResourceCalculator();
Resource capacity = plan.getTotalCapacity();
RLESparseResourceAllocation netRLERes = plan
.getAvailableResourceOverTime(user, oldId, stageArrival, stageDeadline);
RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
user, oldId, stageArrival, stageDeadline, 0);
long step = plan.getStep();

View File

@ -19,7 +19,10 @@
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anySetOf;
import static org.mockito.Mockito.*;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import java.io.FileWriter;
import java.io.IOException;
@ -76,7 +79,8 @@ public static ReservationSchedulerConfiguration createConf(
String reservationQ, long timeWindow, float instConstraint,
float avgConstraint) {
ReservationSchedulerConfiguration realConf = new CapacitySchedulerConfiguration();
ReservationSchedulerConfiguration realConf =
new CapacitySchedulerConfiguration();
ReservationSchedulerConfiguration conf = spy(realConf);
when(conf.getReservationWindow(reservationQ)).thenReturn(timeWindow);
when(conf.getInstantaneousMaxCapacity(reservationQ))
@ -168,7 +172,6 @@ public static FairScheduler setupFairScheduler(RMContext rmContext,
scheduler.start();
scheduler.reinitialize(conf, rmContext);
Resource resource =
ReservationSystemTestUtil.calculateClusterResource(numContainers);
RMNode node1 = MockNodes.newNodeInfo(1, resource, 1, "127.0.0.1");
@ -184,10 +187,16 @@ public static ReservationDefinition createSimpleReservationDefinition(
public static ReservationDefinition createSimpleReservationDefinition(
long arrival, long deadline, long duration, int parallelism) {
return createSimpleReservationDefinition(arrival, deadline, duration,
parallelism, null);
}
public static ReservationDefinition createSimpleReservationDefinition(
long arrival, long deadline, long duration, int parallelism,
String recurrenceExpression) {
// create a request with a single atomic ask
ReservationRequest r =
ReservationRequest.newInstance(Resource.newInstance(1024, 1),
parallelism, parallelism, duration);
ReservationRequest r = ReservationRequest.newInstance(
Resource.newInstance(1024, 1), parallelism, parallelism, duration);
ReservationDefinition rDef = new ReservationDefinitionPBImpl();
ReservationRequests reqs = new ReservationRequestsPBImpl();
reqs.setReservationResources(Collections.singletonList(r));
@ -195,32 +204,31 @@ public static ReservationDefinition createSimpleReservationDefinition(
rDef.setReservationRequests(reqs);
rDef.setArrival(arrival);
rDef.setDeadline(deadline);
if (recurrenceExpression != null) {
rDef.setRecurrenceExpression(recurrenceExpression);
}
return rDef;
}
public static ReservationSubmissionRequest createSimpleReservationRequest(
ReservationId reservationId, int numContainers, long arrival,
long deadline, long duration) {
return createSimpleReservationRequest(reservationId, numContainers,
arrival, deadline, duration, Priority.UNDEFINED);
return createSimpleReservationRequest(reservationId, numContainers, arrival,
deadline, duration, Priority.UNDEFINED);
}
public static ReservationSubmissionRequest createSimpleReservationRequest(
ReservationId reservationId, int numContainers, long arrival,
long deadline, long duration, Priority priority) {
// create a request with a single atomic ask
ReservationRequest r =
ReservationRequest.newInstance(Resource.newInstance(1024, 1),
numContainers, 1, duration);
ReservationRequests reqs =
ReservationRequests.newInstance(Collections.singletonList(r),
ReservationRequestInterpreter.R_ALL);
ReservationDefinition rDef =
ReservationDefinition.newInstance(arrival, deadline, reqs,
"testClientRMService#reservation", "0", priority);
ReservationSubmissionRequest request =
ReservationSubmissionRequest.newInstance(rDef,
reservationQ, reservationId);
ReservationRequest r = ReservationRequest
.newInstance(Resource.newInstance(1024, 1), numContainers, 1, duration);
ReservationRequests reqs = ReservationRequests.newInstance(
Collections.singletonList(r), ReservationRequestInterpreter.R_ALL);
ReservationDefinition rDef = ReservationDefinition.newInstance(arrival,
deadline, reqs, "testClientRMService#reservation", "0", priority);
ReservationSubmissionRequest request = ReservationSubmissionRequest
.newInstance(rDef, reservationQ, reservationId);
return request;
}
@ -252,9 +260,9 @@ public CapacityScheduler mockCapacityScheduler(int numContainers)
return cs;
}
@SuppressWarnings("rawtypes") public static void initializeRMContext(
int numContainers, AbstractYarnScheduler scheduler,
RMContext mockRMContext) {
@SuppressWarnings("rawtypes")
public static void initializeRMContext(int numContainers,
AbstractYarnScheduler scheduler, RMContext mockRMContext) {
when(mockRMContext.getScheduler()).thenReturn(scheduler);
Resource r = calculateClusterResource(numContainers);
@ -262,26 +270,25 @@ public CapacityScheduler mockCapacityScheduler(int numContainers)
}
public static RMContext createRMContext(Configuration conf) {
RMContext mockRmContext = Mockito.spy(
new RMContextImpl(null, null, null, null, null, null,
new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(), null));
RMContext mockRmContext = Mockito.spy(new RMContextImpl(null, null, null,
null, null, null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(), null));
RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
when(nlm.getQueueResource(any(String.class), anySetOf(String.class),
any(Resource.class))).thenAnswer(new Answer<Resource>() {
@Override public Resource answer(InvocationOnMock invocation)
throws Throwable {
Object[] args = invocation.getArguments();
return (Resource) args[2];
}
});
any(Resource.class))).thenAnswer(new Answer<Resource>() {
@Override
public Resource answer(InvocationOnMock invocation) throws Throwable {
Object[] args = invocation.getArguments();
return (Resource) args[2];
}
});
when(nlm.getResourceByLabel(any(String.class), any(Resource.class)))
.thenAnswer(new Answer<Resource>() {
@Override public Resource answer(InvocationOnMock invocation)
throws Throwable {
@Override
public Resource answer(InvocationOnMock invocation) throws Throwable {
Object[] args = invocation.getArguments();
return (Resource) args[1];
}
@ -304,9 +311,8 @@ public static void setupQueueConfiguration(
final String A = CapacitySchedulerConfiguration.ROOT + ".a";
conf.setCapacity(A, 10);
final String dedicated =
CapacitySchedulerConfiguration.ROOT + CapacitySchedulerConfiguration.DOT
+ reservationQ;
final String dedicated = CapacitySchedulerConfiguration.ROOT
+ CapacitySchedulerConfiguration.DOT + reservationQ;
conf.setCapacity(dedicated, 80);
// Set as reservation queue
conf.setReservable(dedicated, true);
@ -405,26 +411,55 @@ public static ReservationDefinition generateRandomRR(Random rand, long i) {
public static Map<ReservationInterval, Resource> generateAllocation(
long startTime, long step, int[] alloc) {
return generateAllocation(startTime, step, alloc, null);
}
public static Map<ReservationInterval, Resource> generateAllocation(
long startTime, long step, int[] alloc, String recurrenceExpression) {
Map<ReservationInterval, Resource> req = new TreeMap<>();
for (int i = 0; i < alloc.length; i++) {
req.put(new ReservationInterval(startTime + i * step,
startTime + (i + 1) * step), ReservationSystemUtil.toResource(
ReservationRequest
.newInstance(Resource.newInstance(1024, 1), alloc[i])));
long period = 0;
if (recurrenceExpression != null) {
period = Long.parseLong(recurrenceExpression);
}
long rStart;
long rEnd;
for (int j = 0; j < 86400000; j += period) {
for (int i = 0; i < alloc.length; i++) {
rStart = (startTime + i * step) + j * period;
rEnd = (startTime + (i + 1) * step) + j * period;
if (period > 0) {
rStart = rStart % period + j * period;
rEnd = rEnd % period + j * period;
if (rStart > rEnd) {
// skip wrap-around entry
continue;
}
}
req.put(new ReservationInterval(rStart, rEnd),
ReservationSystemUtil.toResource(ReservationRequest
.newInstance(Resource.newInstance(1024, 1), alloc[i])));
}
// execute only once if non-periodic
if (period == 0) {
break;
}
}
return req;
}
public static RLESparseResourceAllocation
generateRLESparseResourceAllocation(int[] alloc, long[] timeSteps) {
public static RLESparseResourceAllocation generateRLESparseResourceAllocation(
int[] alloc, long[] timeSteps) {
TreeMap<Long, Resource> allocationsMap = new TreeMap<>();
for (int i = 0; i < alloc.length; i++) {
allocationsMap.put(timeSteps[i],
Resource.newInstance(alloc[i], alloc[i]));
}
RLESparseResourceAllocation rleVector =
new RLESparseResourceAllocation(allocationsMap,
new DefaultResourceCalculator());
RLESparseResourceAllocation rleVector = new RLESparseResourceAllocation(
allocationsMap, new DefaultResourceCalculator());
return rleVector;
}

View File

@ -40,6 +40,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.UTCClock;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.junit.After;
@ -47,6 +48,10 @@
import org.junit.Before;
import org.junit.Test;
/**
* Testing the class {@link InMemoryPlan}.
*/
@SuppressWarnings("checkstyle:nowhitespaceafter")
public class TestInMemoryPlan {
private String user = "yarn";
@ -62,6 +67,7 @@ public class TestInMemoryPlan {
private ReservationAgent agent;
private Planner replanner;
private RMContext context;
private long maxPeriodicity;
@Before
public void setUp() throws PlanningException {
@ -72,7 +78,7 @@ public void setUp() throws PlanningException {
clock = mock(Clock.class);
queueMetrics = mock(QueueMetrics.class);
policy = mock(SharingPolicy.class);
policy = new NoOverCommitPolicy();
replanner = mock(Planner.class);
when(clock.getTime()).thenReturn(1L);
@ -95,15 +101,14 @@ public void tearDown() {
@Test
public void testAddReservation() {
Plan plan =
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
Plan plan = 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;
ReservationAllocation rAllocation = createReservationAllocation
(reservationID, start, alloc);
ReservationAllocation rAllocation =
createReservationAllocation(reservationID, start, alloc);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
@ -111,32 +116,81 @@ public void testAddReservation() {
Assert.fail(e.getMessage());
}
doAssertions(plan, rAllocation);
checkAllocation(plan, alloc, start);
checkAllocation(plan, alloc, start, 0);
}
private void checkAllocation(Plan plan, int[] alloc, int start) {
@Test
public void testAddPeriodicReservation() throws PlanningException {
maxPeriodicity = 100;
Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, maxPeriodicity,
context, new UTCClock());
ReservationId reservationID =
ReservationSystemTestUtil.getNewReservationId();
int[] alloc = { 10, 50 };
int start = 10;
long period = 20;
ReservationAllocation rAllocation = createReservationAllocation(
reservationID, start, alloc, String.valueOf(period));
// use periodicity of 1hr
rAllocation.setPeriodicity(period);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
} catch (PlanningException e) {
Assert.fail(e.getMessage());
}
doAssertions(plan, rAllocation);
checkAllocation(plan, alloc, start, period);
RLESparseResourceAllocation available =
plan.getAvailableResourceOverTime(user, reservationID, 150, 330, 50);
System.out.println(available);
}
private void checkAllocation(Plan plan, int[] alloc, int start,
long periodicity) {
long end = start + alloc.length;
if (periodicity > 0) {
end = end + maxPeriodicity;
}
RLESparseResourceAllocation userCons =
plan.getConsumptionForUserOverTime(user, start, start + alloc.length);
plan.getConsumptionForUserOverTime(user, start, end * 3);
for (int i = 0; i < alloc.length; i++) {
Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
plan.getTotalCommittedResources(start + i));
Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
userCons.getCapacityAtTime(start + i));
// only one instance for non-periodic reservation
if (periodicity <= 0) {
Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
plan.getTotalCommittedResources(start + i));
Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
userCons.getCapacityAtTime(start + i));
} else {
// periodic reservations should repeat
long y = 0;
Resource res = Resource.newInstance(1024 * (alloc[i]), (alloc[i]));
while (y <= end * 2) {
Assert.assertEquals("At time: " + start + i + y, res,
plan.getTotalCommittedResources(start + i + y));
Assert.assertEquals(" At time: " + (start + i + y), res,
userCons.getCapacityAtTime(start + i + y));
y = y + periodicity;
}
}
}
}
@Test
public void testAddEmptyReservation() {
Plan plan =
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
Plan plan = 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;
ReservationAllocation rAllocation = createReservationAllocation
(reservationID, start, alloc);
ReservationAllocation rAllocation =
createReservationAllocation(reservationID, start, alloc);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
@ -148,15 +202,14 @@ public void testAddEmptyReservation() {
@Test
public void testAddReservationAlreadyExists() {
// First add a reservation
Plan plan =
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
Plan plan = 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;
ReservationAllocation rAllocation = createReservationAllocation
(reservationID, start, alloc);
ReservationAllocation rAllocation =
createReservationAllocation(reservationID, start, alloc);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
@ -164,7 +217,7 @@ public void testAddReservationAlreadyExists() {
Assert.fail(e.getMessage());
}
doAssertions(plan, rAllocation);
checkAllocation(plan, alloc, start);
checkAllocation(plan, alloc, start, 0);
// Try to add it again
try {
@ -180,16 +233,15 @@ public void testAddReservationAlreadyExists() {
@Test
public void testUpdateReservation() {
Plan plan =
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
ReservationId reservationID =
ReservationSystemTestUtil.getNewReservationId();
// First add a reservation
int[] alloc = { 10, 10, 10, 10, 10, 10 };
int start = 100;
ReservationAllocation rAllocation = createReservationAllocation
(reservationID, start, alloc);
ReservationAllocation rAllocation =
createReservationAllocation(reservationID, start, alloc);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
@ -210,8 +262,8 @@ public void testUpdateReservation() {
// Now update it
start = 110;
int[] updatedAlloc = { 0, 5, 10, 10, 5, 0 };
rAllocation = createReservationAllocation(reservationID, start,
updatedAlloc, true);
rAllocation =
createReservationAllocation(reservationID, start, updatedAlloc, true);
try {
plan.updateReservation(rAllocation);
} catch (PlanningException e) {
@ -219,32 +271,71 @@ public void testUpdateReservation() {
}
doAssertions(plan, rAllocation);
userCons =
plan.getConsumptionForUserOverTime(user, start, start
+ updatedAlloc.length);
userCons = plan.getConsumptionForUserOverTime(user, start,
start + updatedAlloc.length);
for (int i = 0; i < updatedAlloc.length; i++) {
Assert.assertEquals(
Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
+ i), plan.getTotalCommittedResources(start + i));
Assert.assertEquals(
Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
+ i), userCons.getCapacityAtTime(start + i));
Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
updatedAlloc[i] + i), plan.getTotalCommittedResources(start + i));
Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
updatedAlloc[i] + i), userCons.getCapacityAtTime(start + i));
}
}
@Test
public void testUpdatePeriodicReservation() {
Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
// First add a reservation
ReservationId reservationID =
ReservationSystemTestUtil.getNewReservationId();
int[] alloc = { 10, 20 };
int start = 100;
ReservationAllocation rAllocation =
createReservationAllocation(reservationID, start, alloc);
// use periodicity of 1hr
long period = 3600000;
rAllocation.getReservationDefinition()
.setRecurrenceExpression(String.valueOf(period));
rAllocation.setPeriodicity(period);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
} catch (PlanningException e) {
Assert.fail(e.getMessage());
}
System.out.println(plan.toString());
doAssertions(plan, rAllocation);
checkAllocation(plan, alloc, start, period);
// Now update it
start = 110;
int[] updatedAlloc = { 30, 40, 50 };
rAllocation =
createReservationAllocation(reservationID, start, updatedAlloc);
rAllocation.getReservationDefinition()
.setRecurrenceExpression(String.valueOf(period));
rAllocation.setPeriodicity(period);
try {
plan.updateReservation(rAllocation);
} catch (PlanningException e) {
Assert.fail(e.getMessage());
}
doAssertions(plan, rAllocation);
checkAllocation(plan, updatedAlloc, start, period);
}
@Test
public void testUpdateNonExistingReservation() {
Plan plan =
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
ReservationId reservationID =
ReservationSystemTestUtil.getNewReservationId();
// Try to update a reservation without adding
int[] alloc = { 10, 10, 10, 10, 10, 10 };
int start = 100;
ReservationAllocation rAllocation =
createReservationAllocation(reservationID, start, alloc);
createReservationAllocation(reservationID, start, alloc);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.updateReservation(rAllocation);
@ -260,15 +351,14 @@ public void testUpdateNonExistingReservation() {
@Test
public void testDeleteReservation() {
// First add a reservation
Plan plan =
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
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, true);
createReservationAllocation(reservationID, start, alloc, true);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
@ -306,11 +396,47 @@ public void testDeleteReservation() {
}
}
@Test
public void testDeletePeriodicReservation() {
Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
// First add a reservation
ReservationId reservationID =
ReservationSystemTestUtil.getNewReservationId();
int[] alloc = { 10, 20 };
int start = 100;
ReservationAllocation rAllocation =
createReservationAllocation(reservationID, start, alloc);
// use periodicity of 1hr
long period = 3600000;
rAllocation.getReservationDefinition()
.setRecurrenceExpression(String.valueOf(period));
rAllocation.setPeriodicity(period);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
} catch (PlanningException e) {
Assert.fail(e.getMessage());
}
System.out.println(plan.toString());
doAssertions(plan, rAllocation);
checkAllocation(plan, alloc, start, period);
// Now delete it
try {
plan.deleteReservation(reservationID);
} catch (PlanningException e) {
Assert.fail(e.getMessage());
}
Assert.assertNull(plan.getReservationById(reservationID));
System.out.print(plan);
checkAllocation(plan, new int[] { 0, 0 }, start, period);
}
@Test
public void testDeleteNonExistingReservation() {
Plan plan =
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
ReservationId reservationID =
ReservationSystemTestUtil.getNewReservationId();
// Try to delete a reservation without adding
@ -328,8 +454,9 @@ public void testDeleteNonExistingReservation() {
@Test
public void testArchiveCompletedReservations() {
SharingPolicy sharingPolicy = mock(SharingPolicy.class);
Plan plan =
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
new InMemoryPlan(queueMetrics, sharingPolicy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
ReservationId reservationID1 =
ReservationSystemTestUtil.getNewReservationId();
@ -337,7 +464,7 @@ public void testArchiveCompletedReservations() {
int[] alloc1 = { 10, 10, 10, 10, 10, 10 };
int start = 100;
ReservationAllocation rAllocation =
createReservationAllocation(reservationID1, start, alloc1);
createReservationAllocation(reservationID1, start, alloc1);
Assert.assertNull(plan.getReservationById(reservationID1));
try {
plan.addReservation(rAllocation, false);
@ -345,15 +472,14 @@ public void testArchiveCompletedReservations() {
Assert.fail(e.getMessage());
}
doAssertions(plan, rAllocation);
checkAllocation(plan, alloc1, start);
checkAllocation(plan, alloc1, start, 0);
// Now add another one
ReservationId reservationID2 =
ReservationSystemTestUtil.getNewReservationId();
int[] alloc2 = { 0, 5, 10, 5, 0 };
rAllocation =
createReservationAllocation(reservationID2, start, alloc2, true);
createReservationAllocation(reservationID2, start, alloc2, true);
Assert.assertNull(plan.getReservationById(reservationID2));
try {
plan.addReservation(rAllocation, false);
@ -367,16 +493,18 @@ public void testArchiveCompletedReservations() {
for (int i = 0; i < alloc2.length; i++) {
Assert.assertEquals(
Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
+ alloc2[i] + i), plan.getTotalCommittedResources(start + i));
Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
alloc1[i] + alloc2[i] + i),
plan.getTotalCommittedResources(start + i));
Assert.assertEquals(
Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
+ alloc2[i] + i), userCons.getCapacityAtTime(start + i));
Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
alloc1[i] + alloc2[i] + i),
userCons.getCapacityAtTime(start + i));
}
// Now archive completed reservations
when(clock.getTime()).thenReturn(106L);
when(policy.getValidWindow()).thenReturn(1L);
when(sharingPolicy.getValidWindow()).thenReturn(1L);
try {
// will only remove 2nd reservation as only that has fallen out of the
// archival window
@ -386,7 +514,7 @@ public void testArchiveCompletedReservations() {
}
Assert.assertNotNull(plan.getReservationById(reservationID1));
Assert.assertNull(plan.getReservationById(reservationID2));
checkAllocation(plan, alloc1, start);
checkAllocation(plan, alloc1, start, 0);
when(clock.getTime()).thenReturn(107L);
try {
@ -411,15 +539,14 @@ public void testArchiveCompletedReservations() {
@Test
public void testGetReservationsById() {
Plan plan =
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
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};
ReservationSystemTestUtil.getNewReservationId();
int[] alloc = { 10, 10, 10, 10, 10, 10 };
int start = 100;
ReservationAllocation rAllocation = createReservationAllocation
(reservationID, start, alloc);
ReservationAllocation rAllocation =
createReservationAllocation(reservationID, start, alloc);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
@ -429,31 +556,30 @@ public void testGetReservationsById() {
// Verify that get by reservation id works.
Set<ReservationAllocation> rAllocations =
plan.getReservations(reservationID, null, "");
plan.getReservations(reservationID, null, "");
Assert.assertTrue(rAllocations.size() == 1);
Assert.assertTrue(rAllocation.compareTo(
(ReservationAllocation) rAllocations.toArray()[0]) == 0);
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);
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);
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};
ReservationSystemTestUtil.getNewReservationId();
int[] alloc = { 10, 10, 10, 10, 10, 10 };
int start = 100;
ReservationAllocation rAllocation = createReservationAllocation
(reservationID, start, alloc);
ReservationAllocation rAllocation =
createReservationAllocation(reservationID, start, alloc);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
@ -463,23 +589,22 @@ public void testGetReservationsByInvalidId() {
// If reservationId is null, then nothing is returned.
ReservationId invalidReservationID =
ReservationSystemTestUtil.getNewReservationId();
ReservationSystemTestUtil.getNewReservationId();
Set<ReservationAllocation> rAllocations =
plan.getReservations(invalidReservationID, null, "");
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);
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};
ReservationSystemTestUtil.getNewReservationId();
int[] alloc = { 10, 10, 10, 10, 10, 10 };
int start = 100;
ReservationAllocation rAllocation = createReservationAllocation
(reservationID, start, alloc);
ReservationAllocation rAllocation =
createReservationAllocation(reservationID, start, alloc);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
@ -489,23 +614,24 @@ public void testGetReservationsByTimeInterval() {
// Verify that get by time interval works if the selection interval
// completely overlaps with the allocation.
ReservationInterval interval = new ReservationInterval(rAllocation
.getStartTime(), rAllocation.getEndTime());
ReservationInterval interval = new ReservationInterval(
rAllocation.getStartTime(), rAllocation.getEndTime());
Set<ReservationAllocation> rAllocations =
plan.getReservations(null, interval, "");
plan.getReservations(null, interval, "");
Assert.assertTrue(rAllocations.size() == 1);
Assert.assertTrue(rAllocation.compareTo(
(ReservationAllocation) rAllocations.toArray()[0]) == 0);
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);
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);
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
@ -513,13 +639,13 @@ public void testGetReservationsByTimeInterval() {
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);
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);
interval =
new ReservationInterval(rAllocation.getEndTime(), Long.MAX_VALUE);
rAllocations = plan.getReservations(null, interval, "");
Assert.assertTrue(rAllocations.size() == 0);
@ -532,15 +658,14 @@ public void testGetReservationsByTimeInterval() {
@Test
public void testGetReservationsAtTime() {
Plan plan =
new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
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};
ReservationSystemTestUtil.getNewReservationId();
int[] alloc = { 10, 10, 10, 10, 10, 10 };
int start = 100;
ReservationAllocation rAllocation = createReservationAllocation
(reservationID, start, alloc);
ReservationAllocation rAllocation =
createReservationAllocation(reservationID, start, alloc);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
@ -549,23 +674,22 @@ public void testGetReservationsAtTime() {
}
Set<ReservationAllocation> rAllocations =
plan.getReservationsAtTime(rAllocation.getStartTime());
plan.getReservationsAtTime(rAllocation.getStartTime());
Assert.assertTrue(rAllocations.size() == 1);
Assert.assertTrue(rAllocation.compareTo(
(ReservationAllocation) rAllocations.toArray()[0]) == 0);
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);
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};
ReservationSystemTestUtil.getNewReservationId();
int[] alloc = { 10, 10, 10, 10, 10, 10 };
int start = 100;
ReservationAllocation rAllocation = createReservationAllocation
(reservationID, start, alloc);
ReservationAllocation rAllocation =
createReservationAllocation(reservationID, start, alloc);
Assert.assertNull(plan.getReservationById(reservationID));
try {
plan.addReservation(rAllocation, false);
@ -576,22 +700,21 @@ public void testGetReservationsWithNoInput() {
// Verify that getReservations defaults to getting all reservations if no
// reservationID, time interval, and user is provided,
Set<ReservationAllocation> rAllocations =
plan.getReservations(null, null, "");
plan.getReservations(null, null, "");
Assert.assertTrue(rAllocations.size() == 1);
Assert.assertTrue(rAllocation.compareTo(
(ReservationAllocation) rAllocations.toArray()[0]) == 0);
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);
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, "");
plan.getReservations(null, interval, "");
Assert.assertTrue(rAllocations.size() == 0);
}
@ -600,7 +723,9 @@ private void doAssertions(Plan plan, ReservationAllocation rAllocation) {
Assert.assertNotNull(plan.getReservationById(reservationID));
Assert.assertEquals(rAllocation, plan.getReservationById(reservationID));
Assert.assertTrue(((InMemoryPlan) plan).getAllReservations().size() == 1);
Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
if (rAllocation.getPeriodicity() <= 0) {
Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
}
Assert.assertEquals(totalCapacity, plan.getTotalCapacity());
Assert.assertEquals(minAlloc, plan.getMinimumAllocation());
Assert.assertEquals(maxAlloc, plan.getMaximumAllocation());
@ -610,7 +735,8 @@ private void doAssertions(Plan plan, ReservationAllocation rAllocation) {
}
private ReservationDefinition createSimpleReservationDefinition(long arrival,
long deadline, long duration, Collection<ReservationRequest> resources) {
long deadline, long duration, Collection<ReservationRequest> resources,
String recurrenceExpression) {
// create a request with a single atomic ask
ReservationDefinition rDef = new ReservationDefinitionPBImpl();
ReservationRequests reqs = new ReservationRequestsPBImpl();
@ -619,6 +745,7 @@ private ReservationDefinition createSimpleReservationDefinition(long arrival,
rDef.setReservationRequests(reqs);
rDef.setArrival(arrival);
rDef.setDeadline(deadline);
rDef.setRecurrenceExpression(recurrenceExpression);
return rDef;
}
@ -633,31 +760,43 @@ private Map<ReservationInterval, ReservationRequest> generateAllocation(
} else {
numContainers = alloc[i];
}
ReservationRequest rr =
ReservationRequest.newInstance(Resource.newInstance(1024, 1),
(numContainers));
ReservationRequest rr = ReservationRequest
.newInstance(Resource.newInstance(1024, 1), (numContainers));
req.put(new ReservationInterval(startTime + i, startTime + i + 1), rr);
}
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) {
return createReservationAllocation(reservationID, start, alloc, false, "0");
}
private ReservationAllocation createReservationAllocation(ReservationId
reservationID, int start, int[] alloc, boolean isStep) {
private ReservationAllocation createReservationAllocation(
ReservationId reservationID, int start, int[] alloc, boolean isStep) {
return createReservationAllocation(reservationID, start, alloc, isStep,
"0");
}
private ReservationAllocation createReservationAllocation(
ReservationId reservationID, int start, int[] alloc,
String recurrenceExp) {
return createReservationAllocation(reservationID, start, alloc, false,
recurrenceExp);
}
private ReservationAllocation createReservationAllocation(
ReservationId reservationID, int start, int[] alloc, boolean isStep,
String recurrenceExp) {
Map<ReservationInterval, ReservationRequest> allocations =
generateAllocation(start, alloc, isStep);
generateAllocation(start, alloc, isStep);
ReservationDefinition rDef =
createSimpleReservationDefinition(start, start + alloc.length,
alloc.length, allocations.values());
createSimpleReservationDefinition(start, start + alloc.length,
alloc.length, allocations.values(), recurrenceExp);
Map<ReservationInterval, Resource> allocs =
ReservationSystemUtil.toResources(allocations);
ReservationSystemUtil.toResources(allocations);
return new InMemoryReservationAllocation(reservationID, rDef, user,
planName,
start, start + alloc.length, allocs, resCalc, minAlloc);
planName, start, start + alloc.length, allocs, resCalc, minAlloc);
}
}

View File

@ -19,26 +19,27 @@
package org.apache.hadoop.yarn.server.resourcemanager.reservation;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
import org.junit.Assert;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Testing the class PeriodicRLESparseResourceAllocation.
* Testing the class {@link PeriodicRLESparseResourceAllocation}.
*/
@SuppressWarnings("checkstyle:nowhitespaceafter")
public class TestPeriodicRLESparseResourceAllocation {
private static final Logger LOG = LoggerFactory
.getLogger(TestPeriodicRLESparseResourceAllocation.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestPeriodicRLESparseResourceAllocation.class);
@Test
public void testPeriodicCapacity() {
int[] alloc = {10, 7, 5, 2, 0};
long[] timeSteps = {0L, 5L, 10L, 15L, 19L};
RLESparseResourceAllocation rleSparseVector =
ReservationSystemTestUtil.generateRLESparseResourceAllocation(
alloc, timeSteps);
int[] alloc = { 10, 7, 5, 2, 0 };
long[] timeSteps = { 0L, 5L, 10L, 15L, 19L };
RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
.generateRLESparseResourceAllocation(alloc, timeSteps);
PeriodicRLESparseResourceAllocation periodicVector =
new PeriodicRLESparseResourceAllocation(rleSparseVector, 20L);
LOG.info(periodicVector.toString());
@ -54,43 +55,74 @@ public void testPeriodicCapacity() {
@Test
public void testMaxPeriodicCapacity() {
int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
RLESparseResourceAllocation rleSparseVector =
ReservationSystemTestUtil.generateRLESparseResourceAllocation(
alloc, timeSteps);
int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
.generateRLESparseResourceAllocation(alloc, timeSteps);
PeriodicRLESparseResourceAllocation periodicVector =
new PeriodicRLESparseResourceAllocation(rleSparseVector, 8L);
LOG.info(periodicVector.toString());
Assert.assertEquals(
periodicVector.getMaximumPeriodicCapacity(0, 1),
Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(0, 1),
Resource.newInstance(10, 10));
Assert.assertEquals(
periodicVector.getMaximumPeriodicCapacity(8, 2),
Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(8, 2),
Resource.newInstance(7, 7));
Assert.assertEquals(
periodicVector.getMaximumPeriodicCapacity(16, 3),
Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(16, 3),
Resource.newInstance(10, 10));
Assert.assertEquals(
periodicVector.getMaximumPeriodicCapacity(17, 4),
Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(17, 4),
Resource.newInstance(5, 5));
Assert.assertEquals(
periodicVector.getMaximumPeriodicCapacity(32, 5),
Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(32, 5),
Resource.newInstance(4, 4));
}
@Test
public void testMixPeriodicAndNonPeriodic() throws PlanningException {
int[] alloc = { 2, 5, 0 };
long[] timeSteps = { 1L, 2L, 3L };
RLESparseResourceAllocation tempPeriodic = ReservationSystemTestUtil
.generateRLESparseResourceAllocation(alloc, timeSteps);
PeriodicRLESparseResourceAllocation periodic =
new PeriodicRLESparseResourceAllocation(tempPeriodic, 10L);
int[] alloc2 = { 10, 10, 0 };
long[] timeSteps2 = { 12L, 13L, 14L };
RLESparseResourceAllocation nonPeriodic = ReservationSystemTestUtil
.generateRLESparseResourceAllocation(alloc2, timeSteps2);
RLESparseResourceAllocation merged =
RLESparseResourceAllocation.merge(nonPeriodic.getResourceCalculator(),
Resource.newInstance(100 * 1024, 100), periodic, nonPeriodic,
RLESparseResourceAllocation.RLEOperator.add, 2, 25);
Assert.assertEquals(Resource.newInstance(5, 5),
merged.getCapacityAtTime(2L));
Assert.assertEquals(Resource.newInstance(0, 0),
merged.getCapacityAtTime(3L));
Assert.assertEquals(Resource.newInstance(2, 2),
merged.getCapacityAtTime(11L));
Assert.assertEquals(Resource.newInstance(15, 15),
merged.getCapacityAtTime(12L));
Assert.assertEquals(Resource.newInstance(10, 10),
merged.getCapacityAtTime(13L));
Assert.assertEquals(Resource.newInstance(0, 0),
merged.getCapacityAtTime(14L));
Assert.assertEquals(Resource.newInstance(2, 2),
merged.getCapacityAtTime(21L));
Assert.assertEquals(Resource.newInstance(5, 5),
merged.getCapacityAtTime(22L));
Assert.assertEquals(Resource.newInstance(0, 0),
merged.getCapacityAtTime(23L));
}
@Test
public void testSetCapacityInInterval() {
int[] alloc = {2, 5, 0};
long[] timeSteps = {1L, 2L, 3L};
RLESparseResourceAllocation rleSparseVector =
ReservationSystemTestUtil.generateRLESparseResourceAllocation(
alloc, timeSteps);
int[] alloc = { 2, 5, 0 };
long[] timeSteps = { 1L, 2L, 3L };
RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
.generateRLESparseResourceAllocation(alloc, timeSteps);
PeriodicRLESparseResourceAllocation periodicVector =
new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
ReservationInterval interval = new ReservationInterval(5L, 10L);
periodicVector.addInterval(
interval, Resource.newInstance(8, 8));
periodicVector.addInterval(interval, Resource.newInstance(8, 8));
Assert.assertEquals(Resource.newInstance(8, 8),
periodicVector.getCapacityAtTime(5L));
Assert.assertEquals(Resource.newInstance(8, 8),
@ -99,21 +131,20 @@ public void testSetCapacityInInterval() {
periodicVector.getCapacityAtTime(10L));
Assert.assertEquals(Resource.newInstance(0, 0),
periodicVector.getCapacityAtTime(0L));
Assert.assertFalse(periodicVector.addInterval(
new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
// Assert.assertFalse(periodicVector.addInterval(
// new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
}
public void testRemoveInterval() {
int[] alloc = {2, 5, 3, 4, 0};
long[] timeSteps = {1L, 3L, 5L, 7L, 9L};
RLESparseResourceAllocation rleSparseVector =
ReservationSystemTestUtil.generateRLESparseResourceAllocation(
alloc, timeSteps);
int[] alloc = { 2, 5, 3, 4, 0 };
long[] timeSteps = { 1L, 3L, 5L, 7L, 9L };
RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
.generateRLESparseResourceAllocation(alloc, timeSteps);
PeriodicRLESparseResourceAllocation periodicVector =
new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
ReservationInterval interval = new ReservationInterval(3L, 7L);
Assert.assertTrue(periodicVector.removeInterval(
interval, Resource.newInstance(3, 3)));
Assert.assertTrue(
periodicVector.removeInterval(interval, Resource.newInstance(3, 3)));
Assert.assertEquals(Resource.newInstance(2, 2),
periodicVector.getCapacityAtTime(1L));
Assert.assertEquals(Resource.newInstance(2, 2),

View File

@ -40,10 +40,14 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Testing the class {@link RLESparseResourceAllocation}.
*/
@SuppressWarnings("checkstyle:nowhitespaceafter")
public class TestRLESparseResourceAllocation {
private static final Logger LOG = LoggerFactory
.getLogger(TestRLESparseResourceAllocation.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestRLESparseResourceAllocation.class);
@Test
public void testMergeAdd() throws PlanningException {
@ -196,7 +200,8 @@ public void testMergesubtractTestNonNegative() throws PlanningException {
// Expected!
}
// Testing that the subtractTestNonNegative detects problems even if only one
// Testing that the subtractTestNonNegative detects problems even if only
// one
// of the resource dimensions is "<0"
a.put(10L, Resource.newInstance(10, 5));
b.put(11L, Resource.newInstance(5, 6));
@ -286,9 +291,8 @@ public void testMergeSpeed() throws PlanningException {
public void testRangeOverlapping() {
ResourceCalculator resCalc = new DefaultResourceCalculator();
RLESparseResourceAllocation r =
new RLESparseResourceAllocation(resCalc);
int[] alloc = {10, 10, 10, 10, 10, 10};
RLESparseResourceAllocation r = new RLESparseResourceAllocation(resCalc);
int[] alloc = { 10, 10, 10, 10, 10, 10 };
int start = 100;
Set<Entry<ReservationInterval, Resource>> inputs =
generateAllocation(start, alloc, false).entrySet();
@ -299,9 +303,9 @@ public void testRangeOverlapping() {
long d = r.getLatestNonNullTime();
// tries to trigger "out-of-range" bug
r = r.getRangeOverlapping(s, d);
r = r.getRangeOverlapping(s-1, d-1);
r = r.getRangeOverlapping(s+1, d+1);
r = r.getRangeOverlapping(s, d);
r = r.getRangeOverlapping(s - 1, d - 1);
r = r.getRangeOverlapping(s + 1, d + 1);
}
@Test
@ -370,25 +374,29 @@ public void testPartialRemoval() {
// Current bug prevents this to pass. The RLESparseResourceAllocation
// does not handle removal of "partial"
// allocations correctly.
Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(10)
.getMemorySize());
Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(19).getMemorySize());
Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(21)
.getMemorySize());
Assert.assertEquals(2 * 102400, rleSparseVector.getCapacityAtTime(26)
.getMemorySize());
Assert.assertEquals(102400,
rleSparseVector.getCapacityAtTime(10).getMemorySize());
Assert.assertEquals(0,
rleSparseVector.getCapacityAtTime(13).getMemorySize());
Assert.assertEquals(0,
rleSparseVector.getCapacityAtTime(19).getMemorySize());
Assert.assertEquals(102400,
rleSparseVector.getCapacityAtTime(21).getMemorySize());
Assert.assertEquals(2 * 102400,
rleSparseVector.getCapacityAtTime(26).getMemorySize());
ReservationInterval riRemove2 = new ReservationInterval(9, 13);
rleSparseVector.removeInterval(riRemove2, rr);
LOG.info(rleSparseVector.toString());
Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(11).getMemorySize());
Assert.assertEquals(-102400, rleSparseVector.getCapacityAtTime(9)
.getMemorySize());
Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(20)
.getMemorySize());
Assert.assertEquals(0,
rleSparseVector.getCapacityAtTime(11).getMemorySize());
Assert.assertEquals(-102400,
rleSparseVector.getCapacityAtTime(9).getMemorySize());
Assert.assertEquals(0,
rleSparseVector.getCapacityAtTime(13).getMemorySize());
Assert.assertEquals(102400,
rleSparseVector.getCapacityAtTime(20).getMemorySize());
}
@ -500,7 +508,8 @@ public void testToIntervalMap() {
}
mapAllocations = rleSparseVector.toIntervalMap();
Assert.assertTrue(mapAllocations.size() == 5);
for (Entry<ReservationInterval, Resource> entry : mapAllocations.entrySet()) {
for (Entry<ReservationInterval, Resource> entry : mapAllocations
.entrySet()) {
ReservationInterval interval = entry.getKey();
Resource resource = entry.getValue();
if (interval.getStartTime() == 101L) {
@ -526,59 +535,46 @@ public void testToIntervalMap() {
@Test
public void testMaxPeriodicCapacity() {
long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
RLESparseResourceAllocation rleSparseVector =
ReservationSystemTestUtil.generateRLESparseResourceAllocation(
alloc, timeSteps);
long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
.generateRLESparseResourceAllocation(alloc, timeSteps);
LOG.info(rleSparseVector.toString());
Assert.assertEquals(
rleSparseVector.getMaximumPeriodicCapacity(0, 1),
Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 1),
Resource.newInstance(10, 10));
Assert.assertEquals(
rleSparseVector.getMaximumPeriodicCapacity(0, 2),
Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 2),
Resource.newInstance(7, 7));
Assert.assertEquals(
rleSparseVector.getMaximumPeriodicCapacity(0, 3),
Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 3),
Resource.newInstance(10, 10));
Assert.assertEquals(
rleSparseVector.getMaximumPeriodicCapacity(0, 4),
Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 4),
Resource.newInstance(3, 3));
Assert.assertEquals(
rleSparseVector.getMaximumPeriodicCapacity(0, 5),
Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
Resource.newInstance(4, 4));
Assert.assertEquals(
rleSparseVector.getMaximumPeriodicCapacity(0, 5),
Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
Resource.newInstance(4, 4));
Assert.assertEquals(
rleSparseVector.getMaximumPeriodicCapacity(7, 5),
Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(7, 5),
Resource.newInstance(8, 8));
Assert.assertEquals(
rleSparseVector.getMaximumPeriodicCapacity(10, 3),
Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(10, 3),
Resource.newInstance(0, 0));
}
@Test
public void testGetMinimumCapacityInInterval() {
long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
int[] alloc = {2, 5, 7, 10, 3, 4, 0, 8};
RLESparseResourceAllocation rleSparseVector =
ReservationSystemTestUtil.generateRLESparseResourceAllocation(
alloc, timeSteps);
long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
int[] alloc = { 2, 5, 7, 10, 3, 4, 0, 8 };
RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
.generateRLESparseResourceAllocation(alloc, timeSteps);
LOG.info(rleSparseVector.toString());
Assert.assertEquals(
rleSparseVector.getMinimumCapacityInInterval(
new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
Assert.assertEquals(
rleSparseVector.getMinimumCapacityInInterval(
new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
Assert.assertEquals(
rleSparseVector.getMinimumCapacityInInterval(
new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
}
private void setupArrays(
TreeMap<Long, Resource> a, TreeMap<Long, Resource> b) {
private void setupArrays(TreeMap<Long, Resource> a,
TreeMap<Long, Resource> b) {
a.put(10L, Resource.newInstance(5, 5));
a.put(20L, Resource.newInstance(10, 10));
a.put(30L, Resource.newInstance(15, 15));
@ -620,8 +616,8 @@ private Map<ReservationInterval, Resource> generateAllocation(int startTime,
numContainers = alloc[i];
}
req.put(new ReservationInterval(startTime + i, startTime + i + 1),
ReservationSystemUtil.toResource(ReservationRequest.newInstance(
Resource.newInstance(1024, 1), (numContainers))));
ReservationSystemUtil.toResource(ReservationRequest
.newInstance(Resource.newInstance(1024, 1), (numContainers))));
}
return req;
}

View File

@ -31,6 +31,7 @@
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.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryPlan;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryReservationAllocation;
@ -78,9 +79,10 @@ public void testReplanningPlanCapacityLoss() throws PlanningException {
enf.init("blah", conf);
// Initialize the plan with more resources
InMemoryPlan plan =
new InMemoryPlan(queueMetrics, policy, agent, clusterCapacity, step,
res, minAlloc, maxAlloc, "dedicated", enf, true, context, clock);
InMemoryPlan plan = new InMemoryPlan(queueMetrics, policy, agent,
clusterCapacity, step, res, minAlloc, maxAlloc, "dedicated", enf, true,
YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
context, clock);
// add reservation filling the plan (separating them 1ms, so we are sure
// s2 follows s1 on acceptance