diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 6f64bd1b19d..a470ff10904 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -2346,6 +2346,9 @@ Release 2.6.5 - UNRELEASED YARN-2046. Out of band heartbeats are sent only on container kill and 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 INCOMPATIBLE CHANGES diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java index ee6a6379d7a..f5361c895cf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java @@ -50,6 +50,7 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher { private static final Log LOG = LogFactory.getLog(AsyncDispatcher.class); private final BlockingQueue eventQueue; + private volatile int lastEventQueueSizeLogged = 0; private volatile boolean stopped = false; // 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 */ 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); } int remCapacity = eventQueue.remainingCapacity(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index b2950bbbf96..80b33a38a54 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -727,6 +727,7 @@ public class ResourceManager extends CompositeService implements Recoverable { private final ResourceScheduler scheduler; private final BlockingQueue eventQueue = new LinkedBlockingQueue(); + private volatile int lastEventQueueSizeLogged = 0; private final Thread eventProcessor; private volatile boolean stopped = false; private boolean shouldExitOnError = false; @@ -804,7 +805,9 @@ public class ResourceManager extends CompositeService implements Recoverable { public void handle(SchedulerEvent event) { try { 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); } int remCapacity = eventQueue.remainingCapacity();