|
@@ -179,8 +179,12 @@ public class FairScheduler implements ResourceScheduler {
|
|
|
protected boolean preemptionEnabled;
|
|
|
protected boolean sizeBasedWeight; // Give larger weights to larger jobs
|
|
|
protected WeightAdjuster weightAdjuster; // Can be null for no weight adjuster
|
|
|
+ protected boolean continuousSchedulingEnabled; // Continuous Scheduling enabled or not
|
|
|
+ protected int continuousSchedulingSleepMs; // Sleep time for each pass in continuous scheduling
|
|
|
protected double nodeLocalityThreshold; // Cluster threshold for node locality
|
|
|
protected double rackLocalityThreshold; // Cluster threshold for rack locality
|
|
|
+ protected long nodeLocalityDelayMs; // Delay for node locality
|
|
|
+ protected long rackLocalityDelayMs; // Delay for rack locality
|
|
|
private FairSchedulerEventLog eventLog; // Machine-readable event log
|
|
|
protected boolean assignMultiple; // Allocate multiple containers per
|
|
|
// heartbeat
|
|
@@ -582,6 +586,22 @@ public class FairScheduler implements ResourceScheduler {
|
|
|
return rackLocalityThreshold;
|
|
|
}
|
|
|
|
|
|
+ public long getNodeLocalityDelayMs() {
|
|
|
+ return nodeLocalityDelayMs;
|
|
|
+ }
|
|
|
+
|
|
|
+ public long getRackLocalityDelayMs() {
|
|
|
+ return rackLocalityDelayMs;
|
|
|
+ }
|
|
|
+
|
|
|
+ public boolean isContinuousSchedulingEnabled() {
|
|
|
+ return continuousSchedulingEnabled;
|
|
|
+ }
|
|
|
+
|
|
|
+ public synchronized int getContinuousSchedulingSleepMs() {
|
|
|
+ return continuousSchedulingSleepMs;
|
|
|
+ }
|
|
|
+
|
|
|
public Resource getClusterCapacity() {
|
|
|
return clusterCapacity;
|
|
|
}
|
|
@@ -907,6 +927,37 @@ public class FairScheduler implements ResourceScheduler {
|
|
|
completedContainer, RMContainerEventType.FINISHED);
|
|
|
}
|
|
|
|
|
|
+ if (continuousSchedulingEnabled) {
|
|
|
+ if (!completedContainers.isEmpty()) {
|
|
|
+ attemptScheduling(node);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ attemptScheduling(node);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void continuousScheduling() {
|
|
|
+ while (true) {
|
|
|
+ for (FSSchedulerNode node : nodes.values()) {
|
|
|
+ try {
|
|
|
+ if (Resources.fitsIn(minimumAllocation, node.getAvailableResource())) {
|
|
|
+ attemptScheduling(node);
|
|
|
+ }
|
|
|
+ } catch (Throwable ex) {
|
|
|
+ LOG.warn("Error while attempting scheduling for node " + node + ": " +
|
|
|
+ ex.toString(), ex);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ Thread.sleep(getContinuousSchedulingSleepMs());
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ LOG.warn("Error while doing sleep in continuous scheduling: " +
|
|
|
+ e.toString(), e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private synchronized void attemptScheduling(FSSchedulerNode node) {
|
|
|
// Assign new containers...
|
|
|
// 1. Check for reserved applications
|
|
|
// 2. Schedule if there are no reservations
|
|
@@ -914,19 +965,18 @@ public class FairScheduler implements ResourceScheduler {
|
|
|
AppSchedulable reservedAppSchedulable = node.getReservedAppSchedulable();
|
|
|
if (reservedAppSchedulable != null) {
|
|
|
Priority reservedPriority = node.getReservedContainer().getReservedPriority();
|
|
|
- if (reservedAppSchedulable != null &&
|
|
|
- !reservedAppSchedulable.hasContainerForNode(reservedPriority, node)) {
|
|
|
+ if (!reservedAppSchedulable.hasContainerForNode(reservedPriority, node)) {
|
|
|
// Don't hold the reservation if app can no longer use it
|
|
|
LOG.info("Releasing reservation that cannot be satisfied for application "
|
|
|
+ reservedAppSchedulable.getApp().getApplicationAttemptId()
|
|
|
- + " on node " + nm);
|
|
|
+ + " on node " + node);
|
|
|
reservedAppSchedulable.unreserve(reservedPriority, node);
|
|
|
reservedAppSchedulable = null;
|
|
|
} else {
|
|
|
// Reservation exists; try to fulfill the reservation
|
|
|
LOG.info("Trying to fulfill reservation for application "
|
|
|
+ reservedAppSchedulable.getApp().getApplicationAttemptId()
|
|
|
- + " on node: " + nm);
|
|
|
+ + " on node: " + node);
|
|
|
|
|
|
node.getReservedAppSchedulable().assignReservedContainer(node);
|
|
|
}
|
|
@@ -1060,8 +1110,13 @@ public class FairScheduler implements ResourceScheduler {
|
|
|
maximumAllocation = this.conf.getMaximumAllocation();
|
|
|
incrAllocation = this.conf.getIncrementAllocation();
|
|
|
userAsDefaultQueue = this.conf.getUserAsDefaultQueue();
|
|
|
+ continuousSchedulingEnabled = this.conf.isContinuousSchedulingEnabled();
|
|
|
+ continuousSchedulingSleepMs =
|
|
|
+ this.conf.getContinuousSchedulingSleepMs();
|
|
|
nodeLocalityThreshold = this.conf.getLocalityThresholdNode();
|
|
|
rackLocalityThreshold = this.conf.getLocalityThresholdRack();
|
|
|
+ nodeLocalityDelayMs = this.conf.getLocalityDelayNodeMs();
|
|
|
+ rackLocalityDelayMs = this.conf.getLocalityDelayRackMs();
|
|
|
preemptionEnabled = this.conf.getPreemptionEnabled();
|
|
|
assignMultiple = this.conf.getAssignMultiple();
|
|
|
maxAssign = this.conf.getMaxAssign();
|
|
@@ -1088,6 +1143,21 @@ public class FairScheduler implements ResourceScheduler {
|
|
|
updateThread.setName("FairSchedulerUpdateThread");
|
|
|
updateThread.setDaemon(true);
|
|
|
updateThread.start();
|
|
|
+
|
|
|
+ if (continuousSchedulingEnabled) {
|
|
|
+ // start continuous scheduling thread
|
|
|
+ Thread schedulingThread = new Thread(
|
|
|
+ new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ continuousScheduling();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ );
|
|
|
+ schedulingThread.setName("ContinuousScheduling");
|
|
|
+ schedulingThread.setDaemon(true);
|
|
|
+ schedulingThread.start();
|
|
|
+ }
|
|
|
} else {
|
|
|
try {
|
|
|
queueMgr.reloadAllocs();
|