|
@@ -95,6 +95,9 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
private OrderingPolicy<FiCaSchedulerApp> pendingOrderingPolicy = null;
|
|
private OrderingPolicy<FiCaSchedulerApp> pendingOrderingPolicy = null;
|
|
|
|
|
|
|
|
+ // Always give preference to this while activating the application attempts.
|
|
|
|
+ private OrderingPolicy<FiCaSchedulerApp> pendingOPForRecoveredApps = null;
|
|
|
|
+
|
|
private volatile float minimumAllocationFactor;
|
|
private volatile float minimumAllocationFactor;
|
|
|
|
|
|
private Map<String, User> users = new HashMap<String, User>();
|
|
private Map<String, User> users = new HashMap<String, User>();
|
|
@@ -156,6 +159,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
setOrderingPolicy(conf.<FiCaSchedulerApp>getOrderingPolicy(getQueuePath()));
|
|
setOrderingPolicy(conf.<FiCaSchedulerApp>getOrderingPolicy(getQueuePath()));
|
|
setPendingAppsOrderingPolicy(conf
|
|
setPendingAppsOrderingPolicy(conf
|
|
.<FiCaSchedulerApp> getOrderingPolicy(getQueuePath()));
|
|
.<FiCaSchedulerApp> getOrderingPolicy(getQueuePath()));
|
|
|
|
+ setPendingAppsOrderingPolicyRecovery(conf
|
|
|
|
+ .<FiCaSchedulerApp> getOrderingPolicy(getQueuePath()));
|
|
|
|
|
|
userLimit = conf.getUserLimit(getQueuePath());
|
|
userLimit = conf.getUserLimit(getQueuePath());
|
|
userLimitFactor = conf.getUserLimitFactor(getQueuePath());
|
|
userLimitFactor = conf.getUserLimitFactor(getQueuePath());
|
|
@@ -320,7 +325,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
}
|
|
}
|
|
|
|
|
|
public synchronized int getNumPendingApplications() {
|
|
public synchronized int getNumPendingApplications() {
|
|
- return pendingOrderingPolicy.getNumSchedulableEntities();
|
|
|
|
|
|
+ return pendingOrderingPolicy.getNumSchedulableEntities()
|
|
|
|
+ + pendingOPForRecoveredApps.getNumSchedulableEntities();
|
|
}
|
|
}
|
|
|
|
|
|
public synchronized int getNumActiveApplications() {
|
|
public synchronized int getNumActiveApplications() {
|
|
@@ -599,9 +605,19 @@ public class LeafQueue extends AbstractCSQueue {
|
|
Map<String, Resource> userAmPartitionLimit =
|
|
Map<String, Resource> userAmPartitionLimit =
|
|
new HashMap<String, Resource>();
|
|
new HashMap<String, Resource>();
|
|
|
|
|
|
- for (Iterator<FiCaSchedulerApp> i = getPendingAppsOrderingPolicy()
|
|
|
|
- .getAssignmentIterator(); i.hasNext();) {
|
|
|
|
- FiCaSchedulerApp application = i.next();
|
|
|
|
|
|
+ activateApplications(getPendingAppsOrderingPolicyRecovery()
|
|
|
|
+ .getAssignmentIterator(), amPartitionLimit, userAmPartitionLimit);
|
|
|
|
+
|
|
|
|
+ activateApplications(
|
|
|
|
+ getPendingAppsOrderingPolicy().getAssignmentIterator(),
|
|
|
|
+ amPartitionLimit, userAmPartitionLimit);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private synchronized void activateApplications(
|
|
|
|
+ Iterator<FiCaSchedulerApp> fsApp, Map<String, Resource> amPartitionLimit,
|
|
|
|
+ Map<String, Resource> userAmPartitionLimit) {
|
|
|
|
+ while (fsApp.hasNext()) {
|
|
|
|
+ FiCaSchedulerApp application = fsApp.next();
|
|
ApplicationId applicationId = application.getApplicationId();
|
|
ApplicationId applicationId = application.getApplicationId();
|
|
|
|
|
|
// Get the am-node-partition associated with each application
|
|
// Get the am-node-partition associated with each application
|
|
@@ -692,7 +708,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
metrics.incAMUsed(application.getUser(),
|
|
metrics.incAMUsed(application.getUser(),
|
|
application.getAMResource(partitionName));
|
|
application.getAMResource(partitionName));
|
|
metrics.setAMResouceLimitForUser(application.getUser(), userAMLimit);
|
|
metrics.setAMResouceLimitForUser(application.getUser(), userAMLimit);
|
|
- i.remove();
|
|
|
|
|
|
+ fsApp.remove();
|
|
LOG.info("Application " + applicationId + " from user: "
|
|
LOG.info("Application " + applicationId + " from user: "
|
|
+ application.getUser() + " activated in queue: " + getQueueName());
|
|
+ application.getUser() + " activated in queue: " + getQueueName());
|
|
}
|
|
}
|
|
@@ -702,7 +718,11 @@ public class LeafQueue extends AbstractCSQueue {
|
|
User user) {
|
|
User user) {
|
|
// Accept
|
|
// Accept
|
|
user.submitApplication();
|
|
user.submitApplication();
|
|
- getPendingAppsOrderingPolicy().addSchedulableEntity(application);
|
|
|
|
|
|
+ if (application.isAttemptRecovering()) {
|
|
|
|
+ getPendingAppsOrderingPolicyRecovery().addSchedulableEntity(application);
|
|
|
|
+ } else {
|
|
|
|
+ getPendingAppsOrderingPolicy().addSchedulableEntity(application);
|
|
|
|
+ }
|
|
applicationAttemptMap.put(application.getApplicationAttemptId(), application);
|
|
applicationAttemptMap.put(application.getApplicationAttemptId(), application);
|
|
|
|
|
|
// Activate applications
|
|
// Activate applications
|
|
@@ -742,7 +762,11 @@ public class LeafQueue extends AbstractCSQueue {
|
|
boolean wasActive =
|
|
boolean wasActive =
|
|
orderingPolicy.removeSchedulableEntity(application);
|
|
orderingPolicy.removeSchedulableEntity(application);
|
|
if (!wasActive) {
|
|
if (!wasActive) {
|
|
- pendingOrderingPolicy.removeSchedulableEntity(application);
|
|
|
|
|
|
+ if (application.isAttemptRecovering()) {
|
|
|
|
+ pendingOPForRecoveredApps.removeSchedulableEntity(application);
|
|
|
|
+ } else {
|
|
|
|
+ pendingOrderingPolicy.removeSchedulableEntity(application);
|
|
|
|
+ }
|
|
} else {
|
|
} else {
|
|
queueUsage.decAMUsed(partitionName,
|
|
queueUsage.decAMUsed(partitionName,
|
|
application.getAMResource(partitionName));
|
|
application.getAMResource(partitionName));
|
|
@@ -1491,7 +1515,11 @@ public class LeafQueue extends AbstractCSQueue {
|
|
* Obtain (read-only) collection of pending applications.
|
|
* Obtain (read-only) collection of pending applications.
|
|
*/
|
|
*/
|
|
public Collection<FiCaSchedulerApp> getPendingApplications() {
|
|
public Collection<FiCaSchedulerApp> getPendingApplications() {
|
|
- return pendingOrderingPolicy.getSchedulableEntities();
|
|
|
|
|
|
+ Collection<FiCaSchedulerApp> pendingApps =
|
|
|
|
+ new ArrayList<FiCaSchedulerApp>();
|
|
|
|
+ pendingApps.addAll(pendingOPForRecoveredApps.getSchedulableEntities());
|
|
|
|
+ pendingApps.addAll(pendingOrderingPolicy.getSchedulableEntities());
|
|
|
|
+ return pendingApps;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1535,6 +1563,10 @@ public class LeafQueue extends AbstractCSQueue {
|
|
@Override
|
|
@Override
|
|
public synchronized void collectSchedulerApplications(
|
|
public synchronized void collectSchedulerApplications(
|
|
Collection<ApplicationAttemptId> apps) {
|
|
Collection<ApplicationAttemptId> apps) {
|
|
|
|
+ for (FiCaSchedulerApp pendingApp : pendingOPForRecoveredApps
|
|
|
|
+ .getSchedulableEntities()) {
|
|
|
|
+ apps.add(pendingApp.getApplicationAttemptId());
|
|
|
|
+ }
|
|
for (FiCaSchedulerApp pendingApp : pendingOrderingPolicy
|
|
for (FiCaSchedulerApp pendingApp : pendingOrderingPolicy
|
|
.getSchedulableEntities()) {
|
|
.getSchedulableEntities()) {
|
|
apps.add(pendingApp.getApplicationAttemptId());
|
|
apps.add(pendingApp.getApplicationAttemptId());
|
|
@@ -1670,6 +1702,21 @@ public class LeafQueue extends AbstractCSQueue {
|
|
this.pendingOrderingPolicy = pendingOrderingPolicy;
|
|
this.pendingOrderingPolicy = pendingOrderingPolicy;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public synchronized OrderingPolicy<FiCaSchedulerApp>
|
|
|
|
+ getPendingAppsOrderingPolicyRecovery() {
|
|
|
|
+ return pendingOPForRecoveredApps;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public synchronized void setPendingAppsOrderingPolicyRecovery(
|
|
|
|
+ OrderingPolicy<FiCaSchedulerApp> pendingOrderingPolicyRecovery) {
|
|
|
|
+ if (null != this.pendingOPForRecoveredApps) {
|
|
|
|
+ pendingOrderingPolicyRecovery
|
|
|
|
+ .addAllSchedulableEntities(this.pendingOPForRecoveredApps
|
|
|
|
+ .getSchedulableEntities());
|
|
|
|
+ }
|
|
|
|
+ this.pendingOPForRecoveredApps = pendingOrderingPolicyRecovery;
|
|
|
|
+ }
|
|
|
|
+
|
|
/*
|
|
/*
|
|
* Holds shared values used by all applications in
|
|
* Holds shared values used by all applications in
|
|
* the queue to calculate headroom on demand
|
|
* the queue to calculate headroom on demand
|