YARN-4722. AsyncDispatcher logs redundant event queue sizes (Jason Lowe via sjlee)
(cherry picked from commit 553b591ba0
)
This commit is contained in:
parent
9ea19a861f
commit
432a2367ce
|
@ -2346,6 +2346,9 @@ Release 2.6.5 - UNRELEASED
|
||||||
YARN-2046. Out of band heartbeats are sent only on container kill and
|
YARN-2046. Out of band heartbeats are sent only on container kill and
|
||||||
possibly too early (Ming Ma via jlowe)
|
possibly too early (Ming Ma via jlowe)
|
||||||
|
|
||||||
|
YARN-4722. AsyncDispatcher logs redundant event queue sizes (Jason Lowe via
|
||||||
|
sjlee)
|
||||||
|
|
||||||
Release 2.6.4 - 2016-02-11
|
Release 2.6.4 - 2016-02-11
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
|
|
@ -50,6 +50,7 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
|
||||||
private static final Log LOG = LogFactory.getLog(AsyncDispatcher.class);
|
private static final Log LOG = LogFactory.getLog(AsyncDispatcher.class);
|
||||||
|
|
||||||
private final BlockingQueue<Event> eventQueue;
|
private final BlockingQueue<Event> eventQueue;
|
||||||
|
private volatile int lastEventQueueSizeLogged = 0;
|
||||||
private volatile boolean stopped = false;
|
private volatile boolean stopped = false;
|
||||||
|
|
||||||
// Configuration flag for enabling/disabling draining dispatcher's events on
|
// Configuration flag for enabling/disabling draining dispatcher's events on
|
||||||
|
@ -236,7 +237,9 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
|
||||||
|
|
||||||
/* all this method does is enqueue all the events onto the queue */
|
/* all this method does is enqueue all the events onto the queue */
|
||||||
int qSize = eventQueue.size();
|
int qSize = eventQueue.size();
|
||||||
if (qSize !=0 && qSize %1000 == 0) {
|
if (qSize != 0 && qSize % 1000 == 0
|
||||||
|
&& lastEventQueueSizeLogged != qSize) {
|
||||||
|
lastEventQueueSizeLogged = qSize;
|
||||||
LOG.info("Size of event-queue is " + qSize);
|
LOG.info("Size of event-queue is " + qSize);
|
||||||
}
|
}
|
||||||
int remCapacity = eventQueue.remainingCapacity();
|
int remCapacity = eventQueue.remainingCapacity();
|
||||||
|
|
|
@ -727,6 +727,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
||||||
private final ResourceScheduler scheduler;
|
private final ResourceScheduler scheduler;
|
||||||
private final BlockingQueue<SchedulerEvent> eventQueue =
|
private final BlockingQueue<SchedulerEvent> eventQueue =
|
||||||
new LinkedBlockingQueue<SchedulerEvent>();
|
new LinkedBlockingQueue<SchedulerEvent>();
|
||||||
|
private volatile int lastEventQueueSizeLogged = 0;
|
||||||
private final Thread eventProcessor;
|
private final Thread eventProcessor;
|
||||||
private volatile boolean stopped = false;
|
private volatile boolean stopped = false;
|
||||||
private boolean shouldExitOnError = false;
|
private boolean shouldExitOnError = false;
|
||||||
|
@ -804,7 +805,9 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
||||||
public void handle(SchedulerEvent event) {
|
public void handle(SchedulerEvent event) {
|
||||||
try {
|
try {
|
||||||
int qSize = eventQueue.size();
|
int qSize = eventQueue.size();
|
||||||
if (qSize !=0 && qSize %1000 == 0) {
|
if (qSize != 0 && qSize % 1000 == 0
|
||||||
|
&& lastEventQueueSizeLogged != qSize) {
|
||||||
|
lastEventQueueSizeLogged = qSize;
|
||||||
LOG.info("Size of scheduler event-queue is " + qSize);
|
LOG.info("Size of scheduler event-queue is " + qSize);
|
||||||
}
|
}
|
||||||
int remCapacity = eventQueue.remainingCapacity();
|
int remCapacity = eventQueue.remainingCapacity();
|
||||||
|
|
Loading…
Reference in New Issue