Browse Source

YARN-1297. FairScheduler: Move some logs to debug and check if debug logging is enabled

(cherry picked from commit 4b1dcbbe0c1d0036f65283be4b25d9b2211abed3)
Karthik Kambatla 9 years ago
parent
commit
a5edb45b18

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

@@ -435,7 +435,10 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
   
   
   @Override
   @Override
   public void handle(RMContainerEvent event) {
   public void handle(RMContainerEvent event) {
-    LOG.debug("Processing " + event.getContainerId() + " of type " + event.getType());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing " + event.getContainerId() + " of type " + event
+              .getType());
+    }
     try {
     try {
       writeLock.lock();
       writeLock.lock();
       RMContainerState oldState = getState();
       RMContainerState oldState = getState();

+ 21 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java

@@ -154,11 +154,13 @@ public abstract class SchedulerNode {
 
 
     launchedContainers.put(container.getId(), rmContainer);
     launchedContainers.put(container.getId(), rmContainer);
 
 
-    LOG.info("Assigned container " + container.getId() + " of capacity "
-        + container.getResource() + " on host " + rmNode.getNodeAddress()
-        + ", which has " + numContainers + " containers, "
-        + getAllocatedResource() + " used and " + getUnallocatedResource()
-        + " available after allocation");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Assigned container " + container.getId() + " of capacity "
+              + container.getResource() + " on host " + rmNode.getNodeAddress()
+              + ", which has " + numContainers + " containers, "
+              + getAllocatedResource() + " used and " + getUnallocatedResource()
+              + " available after allocation");
+    }
   }
   }
 
 
   /**
   /**
@@ -175,11 +177,13 @@ public abstract class SchedulerNode {
       addUnallocatedResource(deltaResource);
       addUnallocatedResource(deltaResource);
     }
     }
 
 
-    LOG.info((increase ? "Increased" : "Decreased") + " container "
-        + containerId + " of capacity " + deltaResource + " on host "
-        + rmNode.getNodeAddress() + ", which has " + numContainers
-        + " containers, " + getAllocatedResource() + " used and "
-        + getUnallocatedResource() + " available after allocation");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug((increase ? "Increased" : "Decreased") + " container "
+              + containerId + " of capacity " + deltaResource + " on host "
+              + rmNode.getNodeAddress() + ", which has " + numContainers
+              + " containers, " + getAllocatedResource() + " used and "
+              + getUnallocatedResource() + " available after allocation");
+    }
   }
   }
   
   
   /**
   /**
@@ -261,11 +265,13 @@ public abstract class SchedulerNode {
       updateResource(container);
       updateResource(container);
     }
     }
 
 
-    LOG.info("Released container " + container.getId() + " of capacity "
-        + container.getResource() + " on host " + rmNode.getNodeAddress()
-        + ", which currently has " + numContainers + " containers, "
-        + getAllocatedResource() + " used and " + getUnallocatedResource()
-        + " available" + ", release resources=" + true);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Released container " + container.getId() + " of capacity "
+              + container.getResource() + " on host " + rmNode.getNodeAddress()
+              + ", which currently has " + numContainers + " containers, "
+              + getAllocatedResource() + " used and " + getUnallocatedResource()
+              + " available" + ", release resources=" + true);
+    }
   }
   }
 
 
   /**
   /**

+ 7 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java

@@ -135,12 +135,14 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     rmContainer.handle(
     rmContainer.handle(
         new RMContainerFinishedEvent(
         new RMContainerFinishedEvent(
             containerId,
             containerId,
-            containerStatus, 
+            containerStatus,
             event)
             event)
-        );
-    LOG.info("Completed container: " + rmContainer.getContainerId() + 
-        " in state: " + rmContainer.getState() + " event:" + event);
-    
+    );
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Completed container: " + rmContainer.getContainerId() +
+              " in state: " + rmContainer.getState() + " event:" + event);
+    }
+
     // Remove from the list of containers
     // Remove from the list of containers
     liveContainers.remove(rmContainer.getContainerId());
     liveContainers.remove(rmContainer.getContainerId());
 
 

+ 10 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -494,9 +494,11 @@ public class FairScheduler extends
         // TODO: Not sure if this ever actually adds this to the list of cleanup
         // TODO: Not sure if this ever actually adds this to the list of cleanup
         // containers on the RMNode (see SchedulerNode.releaseContainer()).
         // containers on the RMNode (see SchedulerNode.releaseContainer()).
         super.completedContainer(container, status, RMContainerEventType.KILL);
         super.completedContainer(container, status, RMContainerEventType.KILL);
-        LOG.info("Killing container" + container +
-            " (after waiting for preemption for " +
-            (getClock().getTime() - time) + "ms)");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Killing container" + container +
+                  " (after waiting for preemption for " +
+                  (getClock().getTime() - time) + "ms)");
+        }
       }
       }
     } else {
     } else {
       // track the request in the FSAppAttempt itself
       // track the request in the FSAppAttempt itself
@@ -869,9 +871,11 @@ public class FairScheduler extends
       updateRootQueueMetrics();
       updateRootQueueMetrics();
     }
     }
 
 
-    LOG.info("Application attempt " + application.getApplicationAttemptId()
-        + " released container " + container.getId() + " on node: " + node
-        + " with event: " + event);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Application attempt " + application.getApplicationAttemptId()
+              + " released container " + container.getId() + " on node: " + node
+              + " with event: " + event);
+    }
   }
   }
 
 
   private synchronized void addNode(List<NMContainerStatus> containerReports,
   private synchronized void addNode(List<NMContainerStatus> containerReports,