浏览代码

YARN-4722. AsyncDispatcher logs redundant event queue sizes (Jason Lowe via sjlee)

(cherry picked from commit 553b591ba06bbf0b18dca674d25a48218fed0a26)
Sangjin Lee 9 年之前
父节点
当前提交
84233f013d

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -18,6 +18,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

+ 4 - 1
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
@@ -238,7 +239,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();

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -645,6 +645,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;
@@ -722,7 +723,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();