|
@@ -921,25 +921,25 @@ public class FairScheduler implements ResourceScheduler {
|
|
@Override
|
|
@Override
|
|
public synchronized void reinitialize(Configuration conf, RMContext rmContext)
|
|
public synchronized void reinitialize(Configuration conf, RMContext rmContext)
|
|
throws IOException {
|
|
throws IOException {
|
|
|
|
+ this.conf = new FairSchedulerConfiguration(conf);
|
|
|
|
+ minimumAllocation = this.conf.getMinimumMemoryAllocation();
|
|
|
|
+ maximumAllocation = this.conf.getMaximumMemoryAllocation();
|
|
|
|
+ userAsDefaultQueue = this.conf.getUserAsDefaultQueue();
|
|
|
|
+ nodeLocalityThreshold = this.conf.getLocalityThresholdNode();
|
|
|
|
+ rackLocalityThreshold = this.conf.getLocalityThresholdRack();
|
|
|
|
+ preemptionEnabled = this.conf.getPreemptionEnabled();
|
|
|
|
+ assignMultiple = this.conf.getAssignMultiple();
|
|
|
|
+ maxAssign = this.conf.getMaxAssign();
|
|
|
|
+ sizeBasedWeight = this.conf.getSizeBasedWeight();
|
|
|
|
+
|
|
if (!initialized) {
|
|
if (!initialized) {
|
|
- this.conf = new FairSchedulerConfiguration(conf);
|
|
|
|
rootMetrics = QueueMetrics.forQueue("root", null, true, conf);
|
|
rootMetrics = QueueMetrics.forQueue("root", null, true, conf);
|
|
this.rmContext = rmContext;
|
|
this.rmContext = rmContext;
|
|
this.eventLog = new FairSchedulerEventLog();
|
|
this.eventLog = new FairSchedulerEventLog();
|
|
eventLog.init(this.conf);
|
|
eventLog.init(this.conf);
|
|
- minimumAllocation = this.conf.getMinimumMemoryAllocation();
|
|
|
|
- maximumAllocation = this.conf.getMaximumMemoryAllocation();
|
|
|
|
- userAsDefaultQueue = this.conf.getUserAsDefaultQueue();
|
|
|
|
- nodeLocalityThreshold = this.conf.getLocalityThresholdNode();
|
|
|
|
- rackLocalityThreshold = this.conf.getLocalityThresholdRack();
|
|
|
|
- preemptionEnabled = this.conf.getPreemptionEnabled();
|
|
|
|
- assignMultiple = this.conf.getAssignMultiple();
|
|
|
|
- maxAssign = this.conf.getMaxAssign();
|
|
|
|
|
|
|
|
initialized = true;
|
|
initialized = true;
|
|
|
|
|
|
- sizeBasedWeight = this.conf.getSizeBasedWeight();
|
|
|
|
-
|
|
|
|
try {
|
|
try {
|
|
queueMgr.initialize();
|
|
queueMgr.initialize();
|
|
} catch (Exception e) {
|
|
} catch (Exception e) {
|
|
@@ -951,14 +951,8 @@ public class FairScheduler implements ResourceScheduler {
|
|
updateThread.setDaemon(true);
|
|
updateThread.setDaemon(true);
|
|
updateThread.start();
|
|
updateThread.start();
|
|
} else {
|
|
} else {
|
|
- this.conf = new FairSchedulerConfiguration(conf);
|
|
|
|
- userAsDefaultQueue = this.conf.getUserAsDefaultQueue();
|
|
|
|
- nodeLocalityThreshold = this.conf.getLocalityThresholdNode();
|
|
|
|
- rackLocalityThreshold = this.conf.getLocalityThresholdRack();
|
|
|
|
- preemptionEnabled = this.conf.getPreemptionEnabled();
|
|
|
|
try {
|
|
try {
|
|
queueMgr.reloadAllocs();
|
|
queueMgr.reloadAllocs();
|
|
-
|
|
|
|
} catch (Exception e) {
|
|
} catch (Exception e) {
|
|
throw new IOException("Failed to initialize FairScheduler", e);
|
|
throw new IOException("Failed to initialize FairScheduler", e);
|
|
}
|
|
}
|