Browse Source

YARN-11274. Impove Nodemanager#NodeStatusUpdaterImpl Log. (#4783). Contributed by fanshilun.

Signed-off-by: Ayush Saxena <ayushsaxena@apache.org>
slfan1989 2 years ago
parent
commit
b2760520c3

+ 21 - 30
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java

@@ -201,7 +201,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     // Update configured resources via plugins.
     updateConfiguredResourcesViaPlugins(totalResource);
 
-    LOG.info("Nodemanager resources is set to: " + totalResource);
+    LOG.info("Nodemanager resources is set to: {}.", totalResource);
 
     metrics.addResource(totalResource);
 
@@ -247,9 +247,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     LOG.debug("{} :{}", YARN_NODEMANAGER_DURATION_TO_TRACK_STOPPED_CONTAINERS,
         durationToTrackStoppedContainers);
     super.serviceInit(conf);
-    LOG.info("Initialized nodemanager with :" +
-        " physical-memory=" + memoryMb + " virtual-memory=" + virtualMemoryMb +
-        " virtual-cores=" + virtualCores);
+    LOG.info("Initialized nodemanager with : physical-memory={} virtual-memory={} " +
+        "virtual-cores={}.", memoryMb, virtualMemoryMb, virtualCores);
 
     this.logAggregationEnabled =
         conf.getBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED,
@@ -264,7 +263,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
     // NodeManager is the last service to start, so NodeId is available.
     this.nodeId = this.context.getNodeId();
-    LOG.info("Node ID assigned is : " + this.nodeId);
+    LOG.info("Node ID assigned is : {}.", this.nodeId);
     this.httpPort = this.context.getHttpPort();
     this.nodeManagerVersionId = YarnVersionInfo.getVersion();
     try {
@@ -312,10 +311,9 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     request.setNodeId(this.nodeId);
     try {
       resourceTracker.unRegisterNodeManager(request);
-      LOG.info("Successfully Unregistered the Node " + this.nodeId
-          + " with ResourceManager.");
+      LOG.info("Successfully Unregistered the Node {} with ResourceManager.", this.nodeId);
     } catch (Exception e) {
-      LOG.warn("Unregistration of the Node " + this.nodeId + " failed.", e);
+      LOG.warn("Unregistration of the Node {} failed.", this.nodeId, e);
     }
   }
 
@@ -399,7 +397,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
               nodeLabels, physicalResource, nodeAttributes, nodeStatus);
 
       if (containerReports != null && !containerReports.isEmpty()) {
-        LOG.info("Registering with RM using containers :" + containerReports);
+        LOG.info("Registering with RM using containers.size : {}." + containerReports.size());
       }
       if (logAggregationEnabled) {
         // pull log aggregation status for application running in this NM
@@ -641,6 +639,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         runningApplications.add(appEntry.getKey());
       }
     }
+    LOG.info("Running Applications Size : {}.", runningApplications.size());
     return runningApplications;
   }
 
@@ -667,8 +666,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       }
     }
     if (!containerStatuses.isEmpty()) {
-      LOG.info("Sending out " + containerStatuses.size()
-          + " NM container statuses: " + containerStatuses);
+      LOG.info("Sending out {} container NM container statuses: {}.",
+          containerStatuses.size(), containerStatuses);
     }
     return containerStatuses;
   }
@@ -724,8 +723,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     }
 
     if (!removedContainers.isEmpty()) {
-      LOG.info("Removed completed containers from NM context: "
-          + removedContainers);
+      LOG.info("Removed completed containers from NM context: {}.", removedContainers);
     }
   }
 
@@ -792,7 +790,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
             try {
               context.getNMStateStore().removeContainer(cid);
             } catch (IOException e) {
-              LOG.error("Unable to remove container " + cid + " in store", e);
+              LOG.error("Unable to remove container {} in store.", cid, e);
             }
           }
         }
@@ -839,18 +837,15 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     if (response.getNodeAction() == NodeAction.SHUTDOWN) {
       LOG.warn("Received SHUTDOWN signal from Resourcemanager as part of"
           + " heartbeat, hence shutting down.");
-      LOG.warn("Message from ResourceManager: "
-          + response.getDiagnosticsMessage());
+      LOG.warn("Message from ResourceManager: {}.", response.getDiagnosticsMessage());
       context.setDecommissioned(true);
       dispatcher.getEventHandler().handle(
           new NodeManagerEvent(NodeManagerEventType.SHUTDOWN));
       return true;
     }
     if (response.getNodeAction() == NodeAction.RESYNC) {
-      LOG.warn("Node is out of sync with ResourceManager,"
-          + " hence resyncing.");
-      LOG.warn("Message from ResourceManager: "
-          + response.getDiagnosticsMessage());
+      LOG.warn("Node is out of sync with ResourceManager, hence resyncing.");
+      LOG.warn("Message from ResourceManager: {}.", response.getDiagnosticsMessage());
       // Invalidate the RMIdentifier while resync
       NodeStatusUpdaterImpl.this.rmIdentifier =
           ResourceManagerConstants.RM_INVALID_IDENTIFIER;
@@ -1095,8 +1090,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       try {
         NodeLabelUtil.validateNodeAttributes(nodeAttributes);
       } catch (IOException e) {
-        LOG.error(
-            "Invalid node attribute(s) from Provider : " + e.getMessage());
+        LOG.error("Invalid node attribute(s) from Provider : {}.", e.getMessage());
         throw e;
       }
     }
@@ -1136,9 +1130,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         } else {
           // case where updated node attributes from NodeAttributesProvider
           // is sent to RM and RM rejected the attributes
-          LOG.error("NM node attributes {" + getPreviousValue()
-              + "} were not accepted by RM and message from RM : " + response
-              .getDiagnosticsMessage());
+          LOG.error("NM node attributes [{}] were not accepted by RM and message from RM : {}.",
+              getPreviousValue(), response.getDiagnosticsMessage());
         }
       }
     }
@@ -1262,7 +1255,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         }
       }
       if (hasInvalidLabel) {
-        LOG.error("Invalid Node Label(s) from Provider : " + errorMsg);
+        LOG.error("Invalid Node Label(s) from Provider : {}.", errorMsg);
         throw new IOException(errorMsg.toString());
       }
     }
@@ -1287,10 +1280,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         } else {
           // case where updated labels from NodeLabelsProvider is sent to RM and
           // RM rejected the labels
-          LOG.error(
-              "NM node labels {" + StringUtils.join(",", getPreviousValue())
-                  + "} were not accepted by RM and message from RM : "
-                  + response.getDiagnosticsMessage());
+          LOG.error("NM node labels [{}] were not accepted by RM and message from RM : {}.",
+              StringUtils.join(",", getPreviousValue()), response.getDiagnosticsMessage());
         }
       }
     }

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java

@@ -181,7 +181,7 @@ public class TestNodeStatusUpdater extends NodeManagerTestBase {
         IOException {
       NodeId nodeId = request.getNodeId();
       Resource resource = request.getResource();
-      LOG.info("Registering " + nodeId.toString());
+      LOG.info("Registering {}.", nodeId.toString());
       // NOTE: this really should be checking against the config value
       InetSocketAddress expected = NetUtils.getConnectAddress(
           conf.getSocketAddr(YarnConfiguration.NM_ADDRESS, null, -1));
@@ -217,7 +217,7 @@ public class TestNodeStatusUpdater extends NodeManagerTestBase {
     public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
         throws YarnException, IOException {
       NodeStatus nodeStatus = request.getNodeStatus();
-      LOG.info("Got heartbeat number " + heartBeatID);
+      LOG.info("Got heartbeat number {}.", heartBeatID);
       NodeManagerMetrics mockMetrics = mock(NodeManagerMetrics.class);
       Dispatcher mockDispatcher = mock(Dispatcher.class);
       @SuppressWarnings("unchecked")
@@ -625,7 +625,7 @@ public class TestNodeStatusUpdater extends NodeManagerTestBase {
     @Override
     public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
         throws YarnException, IOException {
-      LOG.info("Got heartBeatId: [" + heartBeatID +"]");
+      LOG.info("Got heartBeatId: [{}]", heartBeatID);
       NodeStatus nodeStatus = request.getNodeStatus();
       nodeStatus.setResponseId(heartBeatID.getAndIncrement());
       NodeHeartbeatResponse nhResponse = YarnServerBuilderUtils.
@@ -644,7 +644,7 @@ public class TestNodeStatusUpdater extends NodeManagerTestBase {
         }
       }
       if (heartBeatID.get() == 2) {
-        LOG.info("Sending FINISH_APP for application: [" + appId + "]");
+        LOG.info("Sending FINISH_APP for application: [{}]", appId);
         this.context.getApplications().put(appId, mock(Application.class));
         nhResponse.addAllApplicationsToCleanup(Collections.singletonList(appId));
       }
@@ -1528,7 +1528,7 @@ public class TestNodeStatusUpdater extends NodeManagerTestBase {
       rt.context.getApplications().remove(rt.appId);
       Assert.assertEquals(1, rt.keepAliveRequests.size());
       int numKeepAliveRequests = rt.keepAliveRequests.get(rt.appId).size();
-      LOG.info("Number of Keep Alive Requests: [" + numKeepAliveRequests + "]");
+      LOG.info("Number of Keep Alive Requests: [{}]", numKeepAliveRequests);
       Assert.assertTrue(numKeepAliveRequests == 2 || numKeepAliveRequests == 3);
       GenericTestUtils.waitFor(
           () -> nm.getServiceState() != STATE.STARTED