Repository: hadoop Updated Branches: refs/heads/branch-2 9ea19a861 -> 432a2367c
YARN-4722. AsyncDispatcher logs redundant event queue sizes (Jason Lowe via sjlee) (cherry picked from commit 553b591ba06bbf0b18dca674d25a48218fed0a26) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/432a2367 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/432a2367 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/432a2367 Branch: refs/heads/branch-2 Commit: 432a2367ce33e4684f74bb04d88e86a6a5aaabbd Parents: 9ea19a8 Author: Sangjin Lee <sj...@apache.org> Authored: Wed Feb 24 09:29:41 2016 -0800 Committer: Sangjin Lee <sj...@apache.org> Committed: Wed Feb 24 09:30:37 2016 -0800 ---------------------------------------------------------------------- hadoop-yarn-project/CHANGES.txt | 3 +++ .../main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java | 5 ++++- .../hadoop/yarn/server/resourcemanager/ResourceManager.java | 5 ++++- 3 files changed, 11 insertions(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/432a2367/hadoop-yarn-project/CHANGES.txt ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 6f64bd1..a470ff1 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 http://git-wip-us.apache.org/repos/asf/hadoop/blob/432a2367/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java ---------------------------------------------------------------------- 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 ee6a637..f5361c8 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<Event> 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(); http://git-wip-us.apache.org/repos/asf/hadoop/blob/432a2367/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java ---------------------------------------------------------------------- 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 b2950bb..80b33a3 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<SchedulerEvent> eventQueue = new LinkedBlockingQueue<SchedulerEvent>(); + 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();