Ver Fonte

YARN-4966. Improve yarn logs to fetch container logs without specifying nodeId. Contributed by Xuan Gong.

(cherry picked from commit 66b07d83740a2ec3e6bfb2bfd064863bae37a1b5)
Varun Vasudev há 9 anos atrás
pai
commit
cf3e93ee73

+ 53 - 30
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java

@@ -212,6 +212,7 @@ public class LogsCLI extends Configured implements Tool {
       appOwner = UserGroupInformation.getCurrentUser().getShortUserName();
     }
 
+    boolean appStateKnown = true;
     YarnApplicationState appState = YarnApplicationState.NEW;
     try {
       appState = getApplicationState(appId);
@@ -222,6 +223,7 @@ public class LogsCLI extends Configured implements Tool {
         return -1;
       }
     } catch (IOException | YarnException e) {
+      appStateKnown = false;
       System.err.println("Unable to get ApplicationState."
           + " Attempting to fetch logs directly from the filesystem.");
     }
@@ -270,7 +272,8 @@ public class LogsCLI extends Configured implements Tool {
     if (containerIdStr != null) {
       // if we provide the node address and the application is in the final
       // state, we could directly get logs from HDFS.
-      if (nodeAddress != null && isApplicationFinished(appState)) {
+      if (nodeAddress != null && (!appStateKnown ||
+          isApplicationFinished(appState))) {
         // if user specified "ALL" as the logFiles param, pass null
         // to logCliHelper so that it fetches all the logs
         List<String> logs;
@@ -284,48 +287,57 @@ public class LogsCLI extends Configured implements Tool {
         return logCliHelper.dumpAContainersLogsForALogType(appIdStr,
             containerIdStr, nodeAddress, appOwner, logs);
       }
+      String nodeHttpAddress = null;
+      String nodeId = null;
       try {
         // If the nodeAddress is not provided, we will try to get
         // the ContainerReport. In the containerReport, we could get
         // nodeAddress and nodeHttpAddress
         ContainerReport report = getContainerReport(containerIdStr);
-        String nodeHttpAddress =
+        nodeHttpAddress =
             report.getNodeHttpAddress().replaceFirst(
               WebAppUtils.getHttpSchemePrefix(getConf()), "");
-        String nodeId = report.getAssignedNode().toString();
-        // If the application is not in the final state,
-        // we will provide the NodeHttpAddress and get the container logs
-        // by calling NodeManager webservice.
-        if (!isApplicationFinished(appState)) {
-          if (logFiles == null || logFiles.length == 0) {
-            logFiles = new String[] { "syslog" };
-          }
-          printContainerLogsFromRunningApplication(getConf(), appIdStr,
-            containerIdStr, nodeHttpAddress, nodeId, logFiles, logCliHelper,
-            appOwner);
-        } else {
+        nodeId = report.getAssignedNode().toString();
+      } catch (IOException | YarnException ex) {
+        if (!appStateKnown || isApplicationFinished(appState)) {
           String [] requestedLogFiles = logFiles;
           if(fetchAllLogFiles(logFiles)) {
             requestedLogFiles = null;
           }
-          // If the application is in the final state, we will directly
-          // get the container logs from HDFS.
-          printContainerLogsForFinishedApplication(appIdStr, containerIdStr,
-            nodeId, requestedLogFiles, logCliHelper, appOwner);
+          return printContainerLogsForFinishedApplicationWithoutNodeId(
+              appIdStr, containerIdStr, requestedLogFiles, logCliHelper,
+              appOwner);
+        } else if (!isApplicationFinished(appState)) {
+          System.err.println("Unable to get logs for this container:"
+              + containerIdStr + "for the application:" + appId);
+          System.out.println("The application: " + appId + " is still running, "
+              + "and we can not get Container report for the container: "
+              + containerIdStr +". Please try later or after the application "
+              + "finishes.");
+          return -1;
         }
-        return resultCode;
-      } catch (IOException | YarnException ex) {
-        System.err.println("Unable to get logs for this container:"
-            + containerIdStr + "for the application:" + appId);
-        if (!getConf().getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
-          YarnConfiguration.DEFAULT_APPLICATION_HISTORY_ENABLED)) {
-          System.out.println("Please enable the application history service. Or ");
+      }
+      // If the application is not in the final state,
+      // we will provide the NodeHttpAddress and get the container logs
+      // by calling NodeManager webservice.
+      if (!isApplicationFinished(appState)) {
+        if (logFiles == null || logFiles.length == 0) {
+          logFiles = new String[] {"syslog"};
         }
-        System.out.println("Using "
-            + "yarn logs -applicationId <appId> -containerId <containerId> "
-            + "--nodeAddress <nodeHttpAddress> to get the container logs");
-        return -1;
+        printContainerLogsFromRunningApplication(getConf(), appIdStr,
+            containerIdStr, nodeHttpAddress, nodeId, logFiles, logCliHelper,
+            appOwner);
+      } else {
+        String[] requestedLogFiles = logFiles;
+        if(fetchAllLogFiles(logFiles)) {
+          requestedLogFiles = null;
+        }
+        // If the application is in the final state, we will directly
+        // get the container logs from HDFS.
+        printContainerLogsForFinishedApplication(appIdStr, containerIdStr,
+            nodeId, requestedLogFiles, logCliHelper, appOwner);
       }
+      return resultCode;
     } else {
       if (nodeAddress == null) {
         resultCode =
@@ -506,7 +518,7 @@ public class LogsCLI extends Configured implements Tool {
     }
     // for the case, we have already uploaded partial logs in HDFS
     logCliHelper.dumpAContainersLogsForALogType(appId, containerIdStr, nodeId,
-      appOwner, Arrays.asList(requestedLogFiles));
+        appOwner, Arrays.asList(requestedLogFiles), false);
   }
 
   private void printContainerLogsForFinishedApplication(String appId,
@@ -519,6 +531,17 @@ public class LogsCLI extends Configured implements Tool {
       nodeAddress, appOwner, logFiles != null ? Arrays.asList(logFiles) : null);
   }
 
+  private int printContainerLogsForFinishedApplicationWithoutNodeId(
+      String appId, String containerId, String[] logFiles,
+      LogCLIHelpers logCliHelper, String appOwner) throws IOException {
+    String containerString = "\n\nContainer: " + containerId;
+    System.out.println(containerString);
+    System.out.println(StringUtils.repeat("=", containerString.length()));
+    return logCliHelper.dumpAContainersLogsForALogTypeWithoutNodeId(appId,
+        containerId, appOwner, logFiles != null ?
+        Arrays.asList(logFiles) : null);
+  }
+
   private ContainerReport getContainerReport(String containerIdStr)
       throws YarnException, IOException {
     YarnClient yarnClient = createYarnClient();

+ 25 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java

@@ -317,6 +317,21 @@ public class TestLogsCLI {
         "Hello container_0_0001_01_000003 in syslog!"));
     sysOutStream.reset();
 
+    YarnClient mockYarnClientWithException =
+        createMockYarnClientWithException();
+    cli = new LogsCLIForTest(mockYarnClientWithException);
+    cli.setConf(configuration);
+
+    exitCode =
+        cli.run(new String[] { "-applicationId", appId.toString(),
+            "-containerId", containerId3.toString() });
+    assertTrue(exitCode == 0);
+    assertTrue(sysOutStream.toString().contains(
+        "Hello container_0_0001_01_000003 in syslog!"));
+    assertTrue(sysOutStream.toString().contains(
+        "Hello container_0_0001_01_000003 in stdout!"));
+    sysOutStream.reset();
+
     fs.delete(new Path(remoteLogRootDir), true);
     fs.delete(new Path(rootLogDir), true);
   }
@@ -439,6 +454,16 @@ public class TestLogsCLI {
     return mockClient;
   }
 
+  private YarnClient createMockYarnClientWithException()
+      throws YarnException, IOException {
+    YarnClient mockClient = mock(YarnClient.class);
+    doThrow(new YarnException()).when(mockClient).getApplicationReport(
+        any(ApplicationId.class));
+    doThrow(new YarnException()).when(mockClient).getContainerReport(
+        any(ContainerId.class));
+    return mockClient;
+  }
+
   private YarnClient createMockYarnClientUnknownApp() throws YarnException,
       IOException {
     YarnClient mockClient = mock(YarnClient.class);

+ 69 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java

@@ -57,7 +57,17 @@ public class LogCLIHelpers implements Configurable {
   @Private
   @VisibleForTesting
   public int dumpAContainersLogsForALogType(String appId, String containerId,
-      String nodeId, String jobOwner, List<String> logType) throws IOException {
+      String nodeId, String jobOwner, List<String> logType)
+      throws IOException {
+    return dumpAContainersLogsForALogType(appId, containerId, nodeId, jobOwner,
+        logType, true);
+  }
+
+  @Private
+  @VisibleForTesting
+  public int dumpAContainersLogsForALogType(String appId, String containerId,
+      String nodeId, String jobOwner, List<String> logType,
+      boolean outputFailure) throws IOException {
     Path remoteRootLogDir = new Path(getConf().get(
         YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
         YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR));
@@ -113,13 +123,70 @@ public class LogCLIHelpers implements Configurable {
         }
       }
     }
-    if (!foundContainerLogs) {
+    if (!foundContainerLogs && outputFailure) {
       containerLogNotFound(containerId);
       return -1;
     }
     return 0;
   }
 
+  @Private
+  public int dumpAContainersLogsForALogTypeWithoutNodeId(String appId,
+      String containerId, String jobOwner, List<String> logType)
+    throws IOException {
+    Path remoteRootLogDir = new Path(getConf().get(
+        YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
+        YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR));
+    ApplicationId applicationId = ConverterUtils.toApplicationId(appId);
+    String user = jobOwner;
+    String logDirSuffix = LogAggregationUtils.getRemoteNodeLogDirSuffix(
+        getConf());
+    Path remoteAppLogDir = LogAggregationUtils.getRemoteAppLogDir(
+        remoteRootLogDir, applicationId, user, logDirSuffix);
+    RemoteIterator<FileStatus> nodeFiles;
+    try {
+      Path qualifiedLogDir =
+          FileContext.getFileContext(getConf()).makeQualified(remoteAppLogDir);
+      nodeFiles = FileContext.getFileContext(qualifiedLogDir.toUri(),
+          getConf()).listStatus(remoteAppLogDir);
+    } catch (FileNotFoundException fnf) {
+      logDirNotExist(remoteAppLogDir.toString());
+      return -1;
+    }
+    boolean foundContainerLogs = false;
+    while(nodeFiles.hasNext()) {
+      FileStatus thisNodeFile = nodeFiles.next();
+      if (!thisNodeFile.getPath().getName().endsWith(
+          LogAggregationUtils.TMP_FILE_SUFFIX)) {
+        AggregatedLogFormat.LogReader reader = null;
+        try {
+          reader =
+              new AggregatedLogFormat.LogReader(getConf(),
+              thisNodeFile.getPath());
+          if (logType == null) {
+            if (dumpAContainerLogs(containerId, reader, System.out,
+                thisNodeFile.getModificationTime()) > -1) {
+              foundContainerLogs = true;
+            }
+          } else {
+            if (dumpAContainerLogsForALogType(containerId, reader, System.out,
+                thisNodeFile.getModificationTime(), logType) > -1) {
+              foundContainerLogs = true;
+            }
+          }
+        } finally {
+          if (reader != null) {
+            reader.close();
+          }
+        }
+      }
+    }
+    if (!foundContainerLogs) {
+      containerLogNotFound(containerId);
+      return -1;
+    }
+    return 0;
+  }
   @Private
   public int dumpAContainerLogs(String containerIdStr,
       AggregatedLogFormat.LogReader reader, PrintStream out,