Просмотр исходного кода

YARN-9592. Use Logger format in ContainersMonitorImpl. Contributed by Inigo Goiri.

Giovanni Matteo Fumarola 6 лет назад
Родитель
Сommit
2210897609

+ 55 - 55
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java

@@ -133,13 +133,12 @@ public class ContainersMonitorImpl extends AbstractService implements
 
     this.resourceCalculatorPlugin =
         ResourceCalculatorPlugin.getContainersMonitorPlugin(this.conf);
-    LOG.info(" Using ResourceCalculatorPlugin : "
-        + this.resourceCalculatorPlugin);
+    LOG.info("Using ResourceCalculatorPlugin: {}",
+        this.resourceCalculatorPlugin);
     processTreeClass = this.conf.getClass(
             YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE, null,
             ResourceCalculatorProcessTree.class);
-    LOG.info(" Using ResourceCalculatorProcessTree : "
-        + this.processTreeClass);
+    LOG.info("Using ResourceCalculatorProcessTree: {}", this.processTreeClass);
 
     this.containerMetricsEnabled =
         this.conf.getBoolean(YarnConfiguration.NM_CONTAINER_METRICS_ENABLE,
@@ -185,10 +184,10 @@ public class ContainersMonitorImpl extends AbstractService implements
     strictMemoryEnforcement = conf.getBoolean(
         YarnConfiguration.NM_MEMORY_RESOURCE_ENFORCED,
         YarnConfiguration.DEFAULT_NM_MEMORY_RESOURCE_ENFORCED);
-    LOG.info("Physical memory check enabled: " + pmemCheckEnabled);
-    LOG.info("Virtual memory check enabled: " + vmemCheckEnabled);
-    LOG.info("Elastic memory control enabled: " + elasticMemoryEnforcement);
-    LOG.info("Strict memory control enabled: " + strictMemoryEnforcement);
+    LOG.info("Physical memory check enabled: {}", pmemCheckEnabled);
+    LOG.info("Virtual memory check enabled: {}", vmemCheckEnabled);
+    LOG.info("Elastic memory control enabled: {}", elasticMemoryEnforcement);
+    LOG.info("Strict memory control enabled: {}", strictMemoryEnforcement);
 
     if (elasticMemoryEnforcement) {
       if (!CGroupElasticMemoryController.isAvailable()) {
@@ -213,7 +212,7 @@ public class ContainersMonitorImpl extends AbstractService implements
 
     containersMonitorEnabled =
         isContainerMonitorEnabled() && monitoringInterval > 0;
-    LOG.info("ContainersMonitor enabled: " + containersMonitorEnabled);
+    LOG.info("ContainersMonitor enabled: {}", containersMonitorEnabled);
 
     nodeCpuPercentageForYARN =
         NodeManagerHardwareUtils.getNodeCpuPercentage(this.conf);
@@ -226,20 +225,21 @@ public class ContainersMonitorImpl extends AbstractService implements
             .getPhysicalMemorySize();
         if (totalPhysicalMemoryOnNM <= 0) {
           LOG.warn("NodeManager's totalPmem could not be calculated. "
-              + "Setting it to " + UNKNOWN_MEMORY_LIMIT);
+              + "Setting it to {}", UNKNOWN_MEMORY_LIMIT);
           totalPhysicalMemoryOnNM = UNKNOWN_MEMORY_LIMIT;
         }
       }
 
       if (totalPhysicalMemoryOnNM != UNKNOWN_MEMORY_LIMIT &&
           this.maxPmemAllottedForContainers > totalPhysicalMemoryOnNM * 0.80f) {
-        LOG.warn("NodeManager configured with "
-            + TraditionalBinaryPrefix.long2String(maxPmemAllottedForContainers,
-                "", 1)
-            + " physical memory allocated to containers, which is more than "
-            + "80% of the total physical memory available ("
-            + TraditionalBinaryPrefix.long2String(totalPhysicalMemoryOnNM, "",
-                1) + "). Thrashing might happen.");
+        LOG.warn(
+            "NodeManager configured with {} physical memory allocated to " +
+            "containers, which is more than 80% of the total physical memory " +
+            "available ({}). Thrashing might happen.",
+            TraditionalBinaryPrefix.long2String(
+                maxPmemAllottedForContainers, "B", 1),
+            TraditionalBinaryPrefix.long2String(
+                totalPhysicalMemoryOnNM, "B", 1));
       }
     }
     super.serviceInit(this.conf);
@@ -264,13 +264,13 @@ public class ContainersMonitorImpl extends AbstractService implements
 
   private boolean isResourceCalculatorAvailable() {
     if (resourceCalculatorPlugin == null) {
-      LOG.info("ResourceCalculatorPlugin is unavailable on this system. " + this
-          .getClass().getName() + " is disabled.");
+      LOG.info("ResourceCalculatorPlugin is unavailable on this system. "
+          + "{} is disabled.", this.getClass().getName());
       return false;
     }
     if (getResourceCalculatorProcessTree("0") == null) {
       LOG.info("ResourceCalculatorProcessTree is unavailable on this system. "
-          + this.getClass().getName() + " is disabled.");
+          + "{} is disabled.", this.getClass().getName());
       return false;
     }
     return true;
@@ -426,15 +426,15 @@ public class ContainersMonitorImpl extends AbstractService implements
     boolean isOverLimit = false;
 
     if (currentMemUsage > (2 * memLimit)) {
-      LOG.warn("Process tree for container: " + containerId
-          + " running over twice " + "the configured limit. Limit=" + memLimit
-          + ", current usage = " + currentMemUsage);
+      LOG.warn("Process tree for container: {} running over twice "
+          + "the configured limit. Limit={}, current usage = {}",
+          containerId, memLimit, currentMemUsage);
       isOverLimit = true;
     } else if (curMemUsageOfAgedProcesses > memLimit) {
-      LOG.warn("Process tree for container: " + containerId
-          + " has processes older than 1 "
-          + "iteration running over the configured limit. Limit=" + memLimit
-          + ", current usage = " + curMemUsageOfAgedProcesses);
+      LOG.warn("Process tree for container: {} has processes older than 1 "
+          + "iteration running over the configured limit. "
+          + "Limit={}, current usage = {}",
+          containerId, memLimit, curMemUsageOfAgedProcesses);
       isOverLimit = true;
     }
 
@@ -468,8 +468,8 @@ public class ContainersMonitorImpl extends AbstractService implements
             tmp.append(p.getPID());
             tmp.append(" ");
           }
-          LOG.debug("Current ProcessTree list : {}",
-              tmp.substring(0, tmp.length()) + "]");
+          tmp.append("]");
+          LOG.debug("Current ProcessTree list : {}", tmp);
         }
 
         // Temporary structure to calculate the total resource utilization of
@@ -495,8 +495,9 @@ public class ContainersMonitorImpl extends AbstractService implements
             if (pId == null || !isResourceCalculatorAvailable()) {
               continue; // processTree cannot be tracked
             }
-            LOG.debug("Constructing ProcessTree for : PID = {}"
-                +" ContainerId = {}", pId, containerId);
+            LOG.debug(
+                "Constructing ProcessTree for : PID = {} ContainerId = {}",
+                pId, containerId);
             ResourceCalculatorProcessTree pTree = ptInfo.getProcessTree();
             pTree.updateProcessTree();    // update process-tree
             long currentVmemUsage = pTree.getVirtualMemorySize();
@@ -509,8 +510,8 @@ public class ContainersMonitorImpl extends AbstractService implements
               // CPU usage is not available likely because the container just
               // started. Let us skip this turn and consider this container
               // in the next iteration.
-              LOG.info("Skipping monitoring container " + containerId
-                  + " since CPU usage is not yet available.");
+              LOG.info("Skipping monitoring container {} since "
+                  + "CPU usage is not yet available.", containerId);
               continue;
             }
 
@@ -558,8 +559,8 @@ public class ContainersMonitorImpl extends AbstractService implements
         try {
           Thread.sleep(monitoringInterval);
         } catch (InterruptedException e) {
-          LOG.warn(ContainersMonitorImpl.class.getName()
-              + " is interrupted. Exiting.");
+          LOG.warn("{} is interrupted. Exiting.",
+              ContainersMonitorImpl.class.getName());
           break;
         }
       }
@@ -604,16 +605,16 @@ public class ContainersMonitorImpl extends AbstractService implements
             if ((ipAndHost != null) && (ipAndHost[0] != null) &&
                 (ipAndHost[1] != null)) {
               container.setIpAndHost(ipAndHost);
-              LOG.info(containerId + "'s ip = " + ipAndHost[0]
-                  + ", and hostname = " + ipAndHost[1]);
+              LOG.info("{}'s ip = {}, and hostname = {}",
+                  containerId, ipAndHost[0], ipAndHost[1]);
             } else {
-              LOG.info("Can not get both ip and hostname: "
-                  + Arrays.toString(ipAndHost));
+              LOG.info("Can not get both ip and hostname: {}",
+                  Arrays.toString(ipAndHost));
             }
             String exposedPorts = containerExecutor.getExposedPorts(container);
             container.setExposedPorts(exposedPorts);
           } else {
-            LOG.info(containerId + " is missing. Not setting ip and hostname");
+            LOG.info("{} is missing. Not setting ip and hostname", containerId);
           }
         }
       }
@@ -648,15 +649,15 @@ public class ContainersMonitorImpl extends AbstractService implements
       long vmemLimit = ptInfo.getVmemLimit();
       long pmemLimit = ptInfo.getPmemLimit();
       if (AUDITLOG.isDebugEnabled()) {
-        AUDITLOG.debug(String.format(
-            "Resource usage of ProcessTree %s for container-id %s:" +
-                " %s CPU:%f CPU/core:%f",
-            pId, containerId.toString(),
+        AUDITLOG.debug(
+            "Resource usage of ProcessTree {} for container-id {}:" +
+            " {} CPU:{} CPU/core:{}",
+            pId, containerId,
             formatUsageString(
                 currentVmemUsage, vmemLimit,
                 currentPmemUsage, pmemLimit),
             cpuUsagePercentPerCore,
-            cpuUsageTotalCoresPercentage));
+            cpuUsageTotalCoresPercentage);
       }
 
       // Add resource utilization for this container
@@ -754,15 +755,15 @@ public class ContainersMonitorImpl extends AbstractService implements
         LOG.warn(msg);
         // warn if not a leader
         if (!pTree.checkPidPgrpidForMatch()) {
-          LOG.error("Killed container process with PID " + pId
-                  + " but it is not a process group leader.");
+          LOG.error("Killed container process with PID {} "
+              + "but it is not a process group leader.", pId);
         }
         // kill the container
         eventDispatcher.getEventHandler().handle(
                 new ContainerKillEvent(containerId,
                       containerExitStatus, msg));
         trackingContainers.remove(containerId);
-        LOG.info("Removed ProcessTree with root " + pId);
+        LOG.info("Removed ProcessTree with root {}", pId);
       }
     }
 
@@ -784,7 +785,7 @@ public class ContainersMonitorImpl extends AbstractService implements
                   currentPmemUsage, cpuUsagePercentPerCore);
         }
       } else {
-        LOG.info(containerId + " does not exist to report");
+        LOG.info("{} does not exist to report", containerId);
       }
     }
 
@@ -967,12 +968,11 @@ public class ContainersMonitorImpl extends AbstractService implements
     if (containersMonitorEnabled) {
       ProcessTreeInfo processTreeInfo = trackingContainers.get(containerId);
       if (processTreeInfo == null) {
-        LOG.warn("Failed to track container "
-            + containerId.toString()
-            + ". It may have already completed.");
+        LOG.warn("Failed to track container {}. It may have already completed.",
+            containerId);
         return;
       }
-      LOG.info("Changing resource-monitoring for " + containerId);
+      LOG.info("Changing resource-monitoring for {}", containerId);
       updateContainerMetrics(monitoringEvent);
       long pmemLimit =
           changeEvent.getResource().getMemorySize() * 1024L * 1024L;
@@ -984,7 +984,7 @@ public class ContainersMonitorImpl extends AbstractService implements
 
   private void onStopMonitoringContainer(
       ContainersMonitorEvent monitoringEvent, ContainerId containerId) {
-    LOG.info("Stopping resource-monitoring for " + containerId);
+    LOG.info("Stopping resource-monitoring for {}", containerId);
     updateContainerMetrics(monitoringEvent);
     trackingContainers.remove(containerId);
   }
@@ -993,7 +993,7 @@ public class ContainersMonitorImpl extends AbstractService implements
       ContainersMonitorEvent monitoringEvent, ContainerId containerId) {
     ContainerStartMonitoringEvent startEvent =
         (ContainerStartMonitoringEvent) monitoringEvent;
-    LOG.info("Starting resource-monitoring for " + containerId);
+    LOG.info("Starting resource-monitoring for {}", containerId);
     updateContainerMetrics(monitoringEvent);
     trackingContainers.put(containerId,
         new ProcessTreeInfo(containerId, null, null,