|
@@ -23,6 +23,9 @@ import java.util.Collection;
|
|
import java.util.Collections;
|
|
import java.util.Collections;
|
|
import java.util.Comparator;
|
|
import java.util.Comparator;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.concurrent.locks.Lock;
|
|
|
|
+import java.util.concurrent.locks.ReadWriteLock;
|
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
@@ -50,6 +53,10 @@ public class FSLeafQueue extends FSQueue {
|
|
new ArrayList<FSAppAttempt>();
|
|
new ArrayList<FSAppAttempt>();
|
|
private final List<FSAppAttempt> nonRunnableApps =
|
|
private final List<FSAppAttempt> nonRunnableApps =
|
|
new ArrayList<FSAppAttempt>();
|
|
new ArrayList<FSAppAttempt>();
|
|
|
|
+ // get a lock with fair distribution for app list updates
|
|
|
|
+ private final ReadWriteLock rwl = new ReentrantReadWriteLock(true);
|
|
|
|
+ private final Lock readLock = rwl.readLock();
|
|
|
|
+ private final Lock writeLock = rwl.writeLock();
|
|
|
|
|
|
private Resource demand = Resources.createResource(0);
|
|
private Resource demand = Resources.createResource(0);
|
|
|
|
|
|
@@ -72,16 +79,26 @@ public class FSLeafQueue extends FSQueue {
|
|
}
|
|
}
|
|
|
|
|
|
public void addApp(FSAppAttempt app, boolean runnable) {
|
|
public void addApp(FSAppAttempt app, boolean runnable) {
|
|
- if (runnable) {
|
|
|
|
- runnableApps.add(app);
|
|
|
|
- } else {
|
|
|
|
- nonRunnableApps.add(app);
|
|
|
|
|
|
+ writeLock.lock();
|
|
|
|
+ try {
|
|
|
|
+ if (runnable) {
|
|
|
|
+ runnableApps.add(app);
|
|
|
|
+ } else {
|
|
|
|
+ nonRunnableApps.add(app);
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ writeLock.unlock();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
// for testing
|
|
// for testing
|
|
void addAppSchedulable(FSAppAttempt appSched) {
|
|
void addAppSchedulable(FSAppAttempt appSched) {
|
|
- runnableApps.add(appSched);
|
|
|
|
|
|
+ writeLock.lock();
|
|
|
|
+ try {
|
|
|
|
+ runnableApps.add(appSched);
|
|
|
|
+ } finally {
|
|
|
|
+ writeLock.unlock();
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -89,18 +106,25 @@ public class FSLeafQueue extends FSQueue {
|
|
* @return whether or not the app was runnable
|
|
* @return whether or not the app was runnable
|
|
*/
|
|
*/
|
|
public boolean removeApp(FSAppAttempt app) {
|
|
public boolean removeApp(FSAppAttempt app) {
|
|
- if (runnableApps.remove(app)) {
|
|
|
|
- // Update AM resource usage
|
|
|
|
- if (app.isAmRunning() && app.getAMResource() != null) {
|
|
|
|
- Resources.subtractFrom(amResourceUsage, app.getAMResource());
|
|
|
|
|
|
+ boolean runnable = false;
|
|
|
|
+ writeLock.lock();
|
|
|
|
+ try {
|
|
|
|
+ if (runnableApps.remove(app)) {
|
|
|
|
+ runnable = true;
|
|
|
|
+ } else if (nonRunnableApps.remove(app)) {
|
|
|
|
+ runnable = false; //nop, runnable is initialised to false already
|
|
|
|
+ } else {
|
|
|
|
+ throw new IllegalStateException("Given app to remove " + app +
|
|
|
|
+ " does not exist in queue " + this);
|
|
}
|
|
}
|
|
- return true;
|
|
|
|
- } else if (nonRunnableApps.remove(app)) {
|
|
|
|
- return false;
|
|
|
|
- } else {
|
|
|
|
- throw new IllegalStateException("Given app to remove " + app +
|
|
|
|
- " does not exist in queue " + this);
|
|
|
|
|
|
+ } finally {
|
|
|
|
+ writeLock.unlock();
|
|
|
|
+ }
|
|
|
|
+ // Update AM resource usage if needed
|
|
|
|
+ if (runnable && app.isAmRunning() && app.getAMResource() != null) {
|
|
|
|
+ Resources.subtractFrom(amResourceUsage, app.getAMResource());
|
|
}
|
|
}
|
|
|
|
+ return runnable;
|
|
}
|
|
}
|
|
|
|
|
|
public Collection<FSAppAttempt> getRunnableAppSchedulables() {
|
|
public Collection<FSAppAttempt> getRunnableAppSchedulables() {
|
|
@@ -114,11 +138,16 @@ public class FSLeafQueue extends FSQueue {
|
|
@Override
|
|
@Override
|
|
public void collectSchedulerApplications(
|
|
public void collectSchedulerApplications(
|
|
Collection<ApplicationAttemptId> apps) {
|
|
Collection<ApplicationAttemptId> apps) {
|
|
- for (FSAppAttempt appSched : runnableApps) {
|
|
|
|
- apps.add(appSched.getApplicationAttemptId());
|
|
|
|
- }
|
|
|
|
- for (FSAppAttempt appSched : nonRunnableApps) {
|
|
|
|
- apps.add(appSched.getApplicationAttemptId());
|
|
|
|
|
|
+ readLock.lock();
|
|
|
|
+ try {
|
|
|
|
+ for (FSAppAttempt appSched : runnableApps) {
|
|
|
|
+ apps.add(appSched.getApplicationAttemptId());
|
|
|
|
+ }
|
|
|
|
+ for (FSAppAttempt appSched : nonRunnableApps) {
|
|
|
|
+ apps.add(appSched.getApplicationAttemptId());
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ readLock.unlock();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -144,11 +173,16 @@ public class FSLeafQueue extends FSQueue {
|
|
@Override
|
|
@Override
|
|
public Resource getResourceUsage() {
|
|
public Resource getResourceUsage() {
|
|
Resource usage = Resources.createResource(0);
|
|
Resource usage = Resources.createResource(0);
|
|
- for (FSAppAttempt app : runnableApps) {
|
|
|
|
- Resources.addTo(usage, app.getResourceUsage());
|
|
|
|
- }
|
|
|
|
- for (FSAppAttempt app : nonRunnableApps) {
|
|
|
|
- Resources.addTo(usage, app.getResourceUsage());
|
|
|
|
|
|
+ readLock.lock();
|
|
|
|
+ try {
|
|
|
|
+ for (FSAppAttempt app : runnableApps) {
|
|
|
|
+ Resources.addTo(usage, app.getResourceUsage());
|
|
|
|
+ }
|
|
|
|
+ for (FSAppAttempt app : nonRunnableApps) {
|
|
|
|
+ Resources.addTo(usage, app.getResourceUsage());
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ readLock.unlock();
|
|
}
|
|
}
|
|
return usage;
|
|
return usage;
|
|
}
|
|
}
|
|
@@ -164,17 +198,22 @@ public class FSLeafQueue extends FSQueue {
|
|
Resource maxRes = scheduler.getAllocationConfiguration()
|
|
Resource maxRes = scheduler.getAllocationConfiguration()
|
|
.getMaxResources(getName());
|
|
.getMaxResources(getName());
|
|
demand = Resources.createResource(0);
|
|
demand = Resources.createResource(0);
|
|
- for (FSAppAttempt sched : runnableApps) {
|
|
|
|
- if (Resources.equals(demand, maxRes)) {
|
|
|
|
- break;
|
|
|
|
|
|
+ readLock.lock();
|
|
|
|
+ try {
|
|
|
|
+ for (FSAppAttempt sched : runnableApps) {
|
|
|
|
+ if (Resources.equals(demand, maxRes)) {
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ updateDemandForApp(sched, maxRes);
|
|
}
|
|
}
|
|
- updateDemandForApp(sched, maxRes);
|
|
|
|
- }
|
|
|
|
- for (FSAppAttempt sched : nonRunnableApps) {
|
|
|
|
- if (Resources.equals(demand, maxRes)) {
|
|
|
|
- break;
|
|
|
|
|
|
+ for (FSAppAttempt sched : nonRunnableApps) {
|
|
|
|
+ if (Resources.equals(demand, maxRes)) {
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ updateDemandForApp(sched, maxRes);
|
|
}
|
|
}
|
|
- updateDemandForApp(sched, maxRes);
|
|
|
|
|
|
+ } finally {
|
|
|
|
+ readLock.unlock();
|
|
}
|
|
}
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("The updated demand for " + getName() + " is " + demand
|
|
LOG.debug("The updated demand for " + getName() + " is " + demand
|
|
@@ -198,7 +237,8 @@ public class FSLeafQueue extends FSQueue {
|
|
public Resource assignContainer(FSSchedulerNode node) {
|
|
public Resource assignContainer(FSSchedulerNode node) {
|
|
Resource assigned = Resources.none();
|
|
Resource assigned = Resources.none();
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
- LOG.debug("Node " + node.getNodeName() + " offered to queue: " + getName());
|
|
|
|
|
|
+ LOG.debug("Node " + node.getNodeName() + " offered to queue: " +
|
|
|
|
+ getName());
|
|
}
|
|
}
|
|
|
|
|
|
if (!assignContainerPreCheck(node)) {
|
|
if (!assignContainerPreCheck(node)) {
|
|
@@ -206,16 +246,26 @@ public class FSLeafQueue extends FSQueue {
|
|
}
|
|
}
|
|
|
|
|
|
Comparator<Schedulable> comparator = policy.getComparator();
|
|
Comparator<Schedulable> comparator = policy.getComparator();
|
|
- Collections.sort(runnableApps, comparator);
|
|
|
|
- for (FSAppAttempt sched : runnableApps) {
|
|
|
|
- if (SchedulerAppUtils.isBlacklisted(sched, node, LOG)) {
|
|
|
|
- continue;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- assigned = sched.assignContainer(node);
|
|
|
|
- if (!assigned.equals(Resources.none())) {
|
|
|
|
- break;
|
|
|
|
|
|
+ writeLock.lock();
|
|
|
|
+ try {
|
|
|
|
+ Collections.sort(runnableApps, comparator);
|
|
|
|
+ } finally {
|
|
|
|
+ writeLock.unlock();
|
|
|
|
+ }
|
|
|
|
+ readLock.lock();
|
|
|
|
+ try {
|
|
|
|
+ for (FSAppAttempt sched : runnableApps) {
|
|
|
|
+ if (SchedulerAppUtils.isBlacklisted(sched, node, LOG)) {
|
|
|
|
+ continue;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ assigned = sched.assignContainer(node);
|
|
|
|
+ if (!assigned.equals(Resources.none())) {
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+ } finally {
|
|
|
|
+ readLock.unlock();
|
|
}
|
|
}
|
|
return assigned;
|
|
return assigned;
|
|
}
|
|
}
|
|
@@ -237,11 +287,16 @@ public class FSLeafQueue extends FSQueue {
|
|
// Choose the app that is most over fair share
|
|
// Choose the app that is most over fair share
|
|
Comparator<Schedulable> comparator = policy.getComparator();
|
|
Comparator<Schedulable> comparator = policy.getComparator();
|
|
FSAppAttempt candidateSched = null;
|
|
FSAppAttempt candidateSched = null;
|
|
- for (FSAppAttempt sched : runnableApps) {
|
|
|
|
- if (candidateSched == null ||
|
|
|
|
- comparator.compare(sched, candidateSched) > 0) {
|
|
|
|
- candidateSched = sched;
|
|
|
|
|
|
+ readLock.lock();
|
|
|
|
+ try {
|
|
|
|
+ for (FSAppAttempt sched : runnableApps) {
|
|
|
|
+ if (candidateSched == null ||
|
|
|
|
+ comparator.compare(sched, candidateSched) > 0) {
|
|
|
|
+ candidateSched = sched;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+ } finally {
|
|
|
|
+ readLock.unlock();
|
|
}
|
|
}
|
|
|
|
|
|
// Preempt from the selected app
|
|
// Preempt from the selected app
|