YARN-8464. Async scheduling thread could be interrupted when there are no NodeManagers in cluster. (Sunil G via wangda)
Change-Id: I4f5f856373378685713e77752ba6cf0988a66065
(cherry picked from commit bedc4fe079
)
This commit is contained in:
parent
8f226f4f1d
commit
ec37e02bc9
|
@ -519,7 +519,14 @@ public class CapacityScheduler extends
|
||||||
// First randomize the start point
|
// First randomize the start point
|
||||||
int current = 0;
|
int current = 0;
|
||||||
Collection<FiCaSchedulerNode> nodes = cs.nodeTracker.getAllNodes();
|
Collection<FiCaSchedulerNode> nodes = cs.nodeTracker.getAllNodes();
|
||||||
int start = random.nextInt(nodes.size());
|
|
||||||
|
// If nodes size is 0 (when there are no node managers registered,
|
||||||
|
// we can return from here itself.
|
||||||
|
int nodeSize = nodes.size();
|
||||||
|
if(nodeSize == 0) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
int start = random.nextInt(nodeSize);
|
||||||
|
|
||||||
// To avoid too verbose DEBUG logging, only print debug log once for
|
// To avoid too verbose DEBUG logging, only print debug log once for
|
||||||
// every 10 secs.
|
// every 10 secs.
|
||||||
|
@ -572,6 +579,7 @@ public class CapacityScheduler extends
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
|
int debuggingLogCounter = 0;
|
||||||
while (!Thread.currentThread().isInterrupted()) {
|
while (!Thread.currentThread().isInterrupted()) {
|
||||||
try {
|
try {
|
||||||
if (!runSchedules.get()) {
|
if (!runSchedules.get()) {
|
||||||
|
@ -583,6 +591,14 @@ public class CapacityScheduler extends
|
||||||
Thread.sleep(1);
|
Thread.sleep(1);
|
||||||
} else{
|
} else{
|
||||||
schedule(cs);
|
schedule(cs);
|
||||||
|
if(LOG.isDebugEnabled()) {
|
||||||
|
// Adding a debug log here to ensure that the thread is alive
|
||||||
|
// and running fine.
|
||||||
|
if (debuggingLogCounter++ > 10000) {
|
||||||
|
debuggingLogCounter = 0;
|
||||||
|
LOG.debug("AsyncScheduleThread[" + getName() + "] is running!");
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (InterruptedException ie) {
|
} catch (InterruptedException ie) {
|
||||||
|
|
Loading…
Reference in New Issue