|
@@ -324,7 +324,7 @@ class CapacitySchedulerQueue {
|
|
|
this.maxJobsToAccept = maxJobsToAccept;
|
|
|
this.maxJobsPerUserToAccept = maxJobsPerUserToAccept;
|
|
|
|
|
|
- LOG.info("Initialized '" + queueName + "' queue with " +
|
|
|
+ LOG.info("Initializing '" + queueName + "' queue with " +
|
|
|
"cap=" + capacityPercent + ", " +
|
|
|
"maxCap=" + maxCapacityPercent + ", " +
|
|
|
"ulMin=" + ulMin + ", " +
|
|
@@ -337,6 +337,14 @@ class CapacitySchedulerQueue {
|
|
|
"maxJobsPerUserToAccept=" + maxJobsPerUserToAccept + ", " +
|
|
|
"maxActiveTasksPerUser=" + maxActiveTasksPerUser
|
|
|
);
|
|
|
+
|
|
|
+ // Sanity checks
|
|
|
+ if (maxActiveTasks < maxActiveTasksPerUser ||
|
|
|
+ maxJobsToInit < maxJobsPerUserToInit ||
|
|
|
+ maxJobsToAccept < maxJobsPerUserToAccept) {
|
|
|
+ throw new IllegalArgumentException("Illegal queue configuration for " +
|
|
|
+ "queue '" + queueName + "'");
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
synchronized void initializeQueue(CapacitySchedulerQueue other) {
|
|
@@ -871,23 +879,31 @@ class CapacitySchedulerQueue {
|
|
|
* the requested number of slots, <code>false</code> otherwise
|
|
|
*/
|
|
|
boolean assignSlotsToJob(TaskType taskType, JobInProgress job, String user) {
|
|
|
+ int numSlotsRequested = job.getNumSlotsPerTask(taskType);
|
|
|
+
|
|
|
// Check to ensure we will not go over the queue's max-capacity
|
|
|
- if (!assignSlotsToQueue(taskType, job.getNumSlotsPerTask(taskType))) {
|
|
|
+ if (!assignSlotsToQueue(taskType, numSlotsRequested)) {
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- // what is our current capacity? It is equal to the queue-capacity if
|
|
|
- // we're running below capacity. If we're running over capacity, then its
|
|
|
- // #running plus slotPerTask of the job (which is the number of extra
|
|
|
- // slots we're getting).
|
|
|
+ // What is our current capacity?
|
|
|
+ // * It is equal to the max(numSlotsRequested queue-capacity) if
|
|
|
+ // we're running below capacity. The 'max' ensures that jobs in queues
|
|
|
+ // with miniscule capacity (< 1 slot) make progress
|
|
|
+ // * If we're running over capacity, then its
|
|
|
+ // #running plus slotPerTask of the job (which is the number of extra
|
|
|
+ // slots we're getting).
|
|
|
+
|
|
|
+ // Allow progress for queues with miniscule capacity
|
|
|
+ int queueCapacity = Math.max(getCapacity(taskType), numSlotsRequested);
|
|
|
+
|
|
|
+ int queueSlotsOccupied = getNumSlotsOccupied(taskType);
|
|
|
int currentCapacity;
|
|
|
- int queueCapacity = getCapacity(taskType);
|
|
|
- if (getNumSlotsOccupied(taskType) < queueCapacity) {
|
|
|
+ if (queueSlotsOccupied < queueCapacity) {
|
|
|
currentCapacity = queueCapacity;
|
|
|
}
|
|
|
else {
|
|
|
- currentCapacity =
|
|
|
- getNumSlotsOccupied(taskType) + job.getNumSlotsPerTask(taskType);
|
|
|
+ currentCapacity = queueSlotsOccupied + numSlotsRequested;
|
|
|
}
|
|
|
|
|
|
// Never allow a single user to take more than the
|
|
@@ -900,11 +916,14 @@ class CapacitySchedulerQueue {
|
|
|
divideAndCeil(ulMin*currentCapacity, 100)),
|
|
|
(int)(queueCapacity * ulMinFactor)
|
|
|
);
|
|
|
- if (getNumSlotsOccupiedByUser(user, taskType) >= limit) {
|
|
|
+
|
|
|
+ if ((getNumSlotsOccupiedByUser(user, taskType) + numSlotsRequested) >
|
|
|
+ limit) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("User " + user + " is over limit, num slots occupied=" +
|
|
|
- getNumSlotsOccupiedByUser(user, taskType) +
|
|
|
- ", limit=" + limit);
|
|
|
+ LOG.debug("User " + user + " is over limit for queue=" + queueName +
|
|
|
+ " num slots occupied=" + getNumSlotsOccupiedByUser(user, taskType) +
|
|
|
+ " limit=" + limit +" numSlotsRequested=" + numSlotsRequested +
|
|
|
+ " currentCapacity=" + currentCapacity);
|
|
|
}
|
|
|
return false;
|
|
|
}
|
|
@@ -952,6 +971,17 @@ class CapacitySchedulerQueue {
|
|
|
"");
|
|
|
}
|
|
|
|
|
|
+ // Task limits - No point accepting the job if it can never be initialized
|
|
|
+ if (job.desiredTasks() > maxActiveTasksPerUser) {
|
|
|
+ throw new IOException(
|
|
|
+ "Job '" + job.getJobID() + "' from user '" + user +
|
|
|
+ "' rejected since it has " + job.desiredTasks() + " tasks which" +
|
|
|
+ " exceeds the limit of " + maxActiveTasksPerUser +
|
|
|
+ " tasks per-user which can be initialized for queue '" +
|
|
|
+ queueName + "'"
|
|
|
+ );
|
|
|
+ }
|
|
|
+
|
|
|
// Across all jobs in queue
|
|
|
if ((getNumWaitingJobs() + getNumRunningJobs()) >= maxJobsToAccept) {
|
|
|
throw new IOException(
|