Przeglądaj źródła

YARN-5089. Improve "yarn log" command-line "logFiles" option to support
regex. Contributed by Xuan Gong

(cherry picked from commit bde819abbbcea940cfa6426a9e4920e6c8dc9cf1)

Xuan 9 lat temu
rodzic
commit
38ae5954df

+ 139 - 169
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java

@@ -23,9 +23,11 @@ import java.io.PrintStream;
 import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
-
+import java.util.Set;
+import java.util.regex.Pattern;
 import javax.ws.rs.core.MediaType;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -53,6 +55,7 @@ import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.logaggregation.LogCLIHelpers;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
@@ -181,9 +184,16 @@ public class LogsCLI extends Configured implements Tool {
       }
     }
 
+    List<String> logs = new ArrayList<String>();
+    if (fetchAllLogFiles(logFiles)) {
+      logs.add(".*");
+    } else if (logFiles != null && logFiles.length > 0) {
+      logs = Arrays.asList(logFiles);
+    }
+
     ContainerLogsRequest request = new ContainerLogsRequest(appId,
-        isApplicationFinished(appState), appOwner,
-        nodeAddress, null, containerIdStr);
+        isApplicationFinished(appState), appOwner, nodeAddress, null,
+        containerIdStr, localDir, logs);
 
     if (showMetaInfo) {
       return showMetaInfo(request, logCliHelper);
@@ -196,7 +206,7 @@ public class LogsCLI extends Configured implements Tool {
     // To get am logs
     if (getAMContainerLogs) {
       return fetchAMContainerLogs(request, amContainersList,
-          logFiles, logCliHelper, localDir);
+          logCliHelper);
     }
 
     int resultCode = 0;
@@ -208,12 +218,10 @@ public class LogsCLI extends Configured implements Tool {
             + " does not have the container:" + containerId);
         return -1;
       }
-      return fetchContainerLogs(request, logFiles,
-          logCliHelper, localDir);
+      return fetchContainerLogs(request, logCliHelper);
     } else {
       if (nodeAddress == null) {
-        resultCode = fetchApplicationLogs(appId, appOwner,
-            logCliHelper, localDir);
+        resultCode = fetchApplicationLogs(request, logCliHelper);
       } else {
         System.err.println("Should at least provide ContainerId!");
         printHelpMessage(printOpts);
@@ -307,14 +315,14 @@ public class LogsCLI extends Configured implements Tool {
   private boolean fetchAllLogFiles(String[] logFiles) {
     if(logFiles != null) {
       List<String> logs = Arrays.asList(logFiles);
-      if(logs.contains("ALL")) {
+      if(logs.contains("ALL") || logs.contains(".*")) {
         return true;
       }
     }
     return false;
   }
 
-  private String[] getContainerLogFiles(Configuration conf,
+  private List<String> getContainerLogFiles(Configuration conf,
       String containerIdStr, String nodeHttpAddress) throws IOException {
     List<String> logFiles = new ArrayList<>();
     Client webServiceClient = Client.create();
@@ -348,32 +356,37 @@ public class LogsCLI extends Configured implements Tool {
       System.err.println("Unable to fetch log files list");
       throw new IOException(ex);
     }
-    return logFiles.toArray(new String[0]);
+    return logFiles;
   }
 
   private void printContainerLogsFromRunningApplication(Configuration conf,
-      ContainerLogsRequest request, String[] logFiles,
-      LogCLIHelpers logCliHelper, String localDir) throws IOException {
-    String appId = request.getAppId().toString();
+      ContainerLogsRequest request, LogCLIHelpers logCliHelper)
+      throws IOException {
     String containerIdStr = request.getContainerId().toString();
-    String[] requestedLogFiles = logFiles;
+    String localDir = request.getOutputLocalDir();
     String nodeHttpAddress = request.getNodeHttpAddress();
     String nodeId = request.getNodeId();
-    String appOwner = request.getAppOwner();
     PrintStream out = logCliHelper.createPrintStream(localDir, nodeId,
         containerIdStr);
     try {
       // fetch all the log files for the container
-      if (fetchAllLogFiles(logFiles)) {
-        requestedLogFiles =
-            getContainerLogFiles(getConf(), containerIdStr, nodeHttpAddress);
+      // filter the log files based on the given --logFiles pattern
+      List<String> allLogs=
+          getContainerLogFiles(getConf(), containerIdStr, nodeHttpAddress);
+      List<String> matchedFiles = getMatchedLogFiles(
+          request, allLogs, true);
+      if (matchedFiles.isEmpty()) {
+        return;
       }
+      ContainerLogsRequest newOptions = new ContainerLogsRequest(request);
+      newOptions.setLogTypes(matchedFiles);
+
       Client webServiceClient = Client.create();
       String containerString = "\n\nContainer: " + containerIdStr;
       out.println(containerString);
       out.println(StringUtils.repeat("=", containerString.length()));
 
-      for (String logFile : requestedLogFiles) {
+      for (String logFile : newOptions.getLogTypes()) {
         out.println("LogType:" + logFile);
         out.println("Log Upload Time:"
             + Times.format(System.currentTimeMillis()));
@@ -395,31 +408,34 @@ public class LogsCLI extends Configured implements Tool {
               + nodeId);
         }
       }
+      // for the case, we have already uploaded partial logs in HDFS
+      logCliHelper.dumpAContainersLogsForALogType(newOptions, false);
     } finally {
       logCliHelper.closePrintStream(out);
     }
-    // for the case, we have already uploaded partial logs in HDFS
-    logCliHelper.dumpAContainersLogsForALogType(appId, containerIdStr, nodeId,
-        appOwner, Arrays.asList(requestedLogFiles), false, localDir);
   }
 
-  private void printContainerLogsForFinishedApplication(
-      ContainerLogsRequest request, String[] logFiles,
-      LogCLIHelpers logCliHelper, String localDir)
+  private int printContainerLogsForFinishedApplication(
+      ContainerLogsRequest request, LogCLIHelpers logCliHelper)
       throws IOException {
-    logCliHelper.dumpAContainersLogsForALogType(request.getAppId().toString(),
-        request.getContainerId().toString(), request.getNodeId(),
-        request.getAppOwner(), logFiles != null ? Arrays.asList(logFiles)
-        : null, localDir);
+    ContainerLogsRequest newOptions = getMatchedLogOptions(
+        request, logCliHelper);
+    if (newOptions == null) {
+      return -1;
+    }
+    return logCliHelper.dumpAContainersLogsForALogType(newOptions);
   }
 
   private int printContainerLogsForFinishedApplicationWithoutNodeId(
-      String appId, String containerId, String[] logFiles,
-      LogCLIHelpers logCliHelper, String appOwner, String localDir)
+      ContainerLogsRequest request, LogCLIHelpers logCliHelper)
       throws IOException {
-    return logCliHelper.dumpAContainersLogsForALogTypeWithoutNodeId(appId,
-        containerId, appOwner, logFiles != null ?
-        Arrays.asList(logFiles) : null, localDir);
+    ContainerLogsRequest newOptions = getMatchedLogOptions(
+        request, logCliHelper);
+    if (newOptions == null) {
+      return -1;
+    }
+    return logCliHelper.dumpAContainersLogsForALogTypeWithoutNodeId(
+        newOptions);
   }
 
   private ContainerReport getContainerReport(String containerIdStr)
@@ -441,8 +457,7 @@ public class LogsCLI extends Configured implements Tool {
 
   private int printAMContainerLogs(Configuration conf,
       ContainerLogsRequest request, List<String> amContainers,
-      String[] logFiles, LogCLIHelpers logCliHelper, String localDir)
-      throws Exception {
+      LogCLIHelpers logCliHelper) throws Exception {
     List<JSONObject> amContainersList = null;
     List<ContainerLogsRequest> requests =
         new ArrayList<ContainerLogsRequest>();
@@ -491,8 +506,7 @@ public class LogsCLI extends Configured implements Tool {
 
     if (amContainers.contains("ALL")) {
       for (ContainerLogsRequest amRequest : requests) {
-        outputAMContainerLogs(amRequest, conf, logFiles,
-            logCliHelper, localDir);
+        outputAMContainerLogs(amRequest, conf, logCliHelper);
       }
       System.out.println();      
       System.out.println("Specified ALL for -am option. "
@@ -502,11 +516,11 @@ public class LogsCLI extends Configured implements Tool {
         int amContainerId = Integer.parseInt(amContainer.trim());
         if (amContainerId == -1) {
           outputAMContainerLogs(requests.get(requests.size() - 1), conf,
-              logFiles, logCliHelper, localDir);
+              logCliHelper);
         } else {
           if (amContainerId <= requests.size()) {
             outputAMContainerLogs(requests.get(amContainerId - 1), conf,
-                logFiles, logCliHelper, localDir);
+                logCliHelper);
           }
         }
       }
@@ -515,8 +529,7 @@ public class LogsCLI extends Configured implements Tool {
   }
 
   private void outputAMContainerLogs(ContainerLogsRequest request,
-      Configuration conf, String[] logFiles,
-      LogCLIHelpers logCliHelper, String localDir) throws Exception {
+      Configuration conf, LogCLIHelpers logCliHelper) throws Exception {
     String nodeHttpAddress = request.getNodeHttpAddress();
     String containerId = request.getContainerId();
     String nodeId = request.getNodeId();
@@ -534,25 +547,15 @@ public class LogsCLI extends Configured implements Tool {
           }
         }
         if (nodeId != null && !nodeId.isEmpty()) {
-          String[] requestedLogFilesList = null;
-          if(!fetchAllLogFiles(logFiles)) {
-            requestedLogFilesList = logFiles;
-          }
           printContainerLogsForFinishedApplication(request,
-              requestedLogFilesList, logCliHelper, localDir);
+              logCliHelper);
         }
       }
     } else {
       if (nodeHttpAddress != null && containerId != null
           && !nodeHttpAddress.isEmpty() && !containerId.isEmpty()) {
-        String[] requestedLogFiles = logFiles;
-        // fetch all the log files for the AM
-        if (fetchAllLogFiles(logFiles)) {
-          requestedLogFiles =
-              getContainerLogFiles(getConf(), containerId, nodeHttpAddress);
-        }
         printContainerLogsFromRunningApplication(conf,
-            request, requestedLogFiles, logCliHelper, localDir);
+            request, logCliHelper);
       }
     }
   }
@@ -564,9 +567,7 @@ public class LogsCLI extends Configured implements Tool {
           + "with finished applications");
       return -1;
     } else {
-      logCliHelper.printLogMetadata(request.getAppId(),
-          request.getContainerId(), request.getNodeId(),
-          request.getAppOwner(), System.out, System.err);
+      logCliHelper.printLogMetadata(request, System.out, System.err);
       return 0;
     }
   }
@@ -578,8 +579,7 @@ public class LogsCLI extends Configured implements Tool {
           + "finished applications");
       return -1;
     } else {
-      logCliHelper.printNodesList(request.getAppId(), request.getAppOwner(),
-          System.out, System.err);
+      logCliHelper.printNodesList(request, System.out, System.err);
       return 0;
     }
   }
@@ -614,7 +614,7 @@ public class LogsCLI extends Configured implements Tool {
     Option logFileOpt = new Option(CONTAINER_LOG_FILES, true,
         "Work with -am/-containerId and specify comma-separated value "
         + "to get specified container log files. Use \"ALL\" to fetch all the "
-        + "log files for the container.");
+        + "log files for the container. It also supports Java Regex.");
     logFileOpt.setValueSeparator(',');
     logFileOpt.setArgs(Option.UNLIMITED_VALUES);
     logFileOpt.setArgName("Log File Name");
@@ -687,13 +687,15 @@ public class LogsCLI extends Configured implements Tool {
   }
 
   private int fetchAMContainerLogs(ContainerLogsRequest request,
-      List<String> amContainersList, String[] logFiles,
-      LogCLIHelpers logCliHelper, String localDir) throws Exception {
+      List<String> amContainersList, LogCLIHelpers logCliHelper)
+      throws Exception {
+    List<String> logFiles = request.getLogTypes();
     // if we do not specify the value for CONTAINER_LOG_FILES option,
     // we will only output syslog
-    if (logFiles == null || logFiles.length == 0) {
-      logFiles = new String[] {"syslog"};
+    if (logFiles == null || logFiles.isEmpty()) {
+      logFiles = Arrays.asList("syslog");
     }
+    request.setLogTypes(logFiles);
     // If the application is running, we will call the RM WebService
     // to get the AppAttempts which includes the nodeHttpAddress
     // and containerId for all the AM Containers.
@@ -701,7 +703,7 @@ public class LogsCLI extends Configured implements Tool {
     // related logs
     if (!request.isAppFinished()) {
       return printAMContainerLogs(getConf(), request, amContainersList,
-          logFiles, logCliHelper, localDir);
+          logCliHelper);
     } else {
       // If the application is in the final state, we will call RM webservice
       // to get all AppAttempts information first. If we get nothing,
@@ -712,7 +714,7 @@ public class LogsCLI extends Configured implements Tool {
       if (getConf().getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
           YarnConfiguration.DEFAULT_APPLICATION_HISTORY_ENABLED)) {
         return printAMContainerLogs(getConf(), request, amContainersList,
-            logFiles, logCliHelper, localDir);
+            logCliHelper);
       } else {
         ApplicationId appId = request.getAppId();
         String appOwner = request.getAppOwner();
@@ -729,29 +731,21 @@ public class LogsCLI extends Configured implements Tool {
   }
 
   private int fetchContainerLogs(ContainerLogsRequest request,
-      String[] logFiles, LogCLIHelpers logCliHelper, String localDir)
-      throws IOException {
+      LogCLIHelpers logCliHelper) throws IOException {
     int resultCode = 0;
     String appIdStr = request.getAppId().toString();
     String containerIdStr = request.getContainerId();
     String nodeAddress = request.getNodeId();
     String appOwner = request.getAppOwner();
     boolean isAppFinished = request.isAppFinished();
+    List<String> logFiles = request.getLogTypes();
     // if we provide the node address and the application is in the final
     // state, we could directly get logs from HDFS.
     if (nodeAddress != null && isAppFinished) {
-      // if user specified "ALL" as the logFiles param, pass null
+      // if user specified "ALL" as the logFiles param, pass empty list
       // to logCliHelper so that it fetches all the logs
-      List<String> logs;
-      if (logFiles == null) {
-        logs = null;
-      } else if (fetchAllLogFiles(logFiles)) {
-        logs = null;
-      } else {
-        logs = Arrays.asList(logFiles);
-      }
-      return logCliHelper.dumpAContainersLogsForALogType(appIdStr,
-          containerIdStr, nodeAddress, appOwner, logs, localDir);
+      return printContainerLogsForFinishedApplication(
+          request, logCliHelper);
     }
     String nodeHttpAddress = null;
     String nodeId = null;
@@ -768,13 +762,8 @@ public class LogsCLI extends Configured implements Tool {
       request.setNodeHttpAddress(nodeHttpAddress);
     } catch (IOException | YarnException ex) {
       if (isAppFinished) {
-        String[] requestedLogFiles = logFiles;
-        if(fetchAllLogFiles(logFiles)) {
-          requestedLogFiles = null;
-        }
         return printContainerLogsForFinishedApplicationWithoutNodeId(
-            appIdStr, containerIdStr, requestedLogFiles, logCliHelper,
-            appOwner, localDir);
+            request, logCliHelper);
       } else {
         System.err.println("Unable to get logs for this container:"
             + containerIdStr + "for the application:" + appIdStr
@@ -790,31 +779,39 @@ public class LogsCLI extends Configured implements Tool {
     // we will provide the NodeHttpAddress and get the container logs
     // by calling NodeManager webservice.
     if (!isAppFinished) {
-      if (logFiles == null || logFiles.length == 0) {
-        logFiles = new String[] {"syslog"};
+      // if we do not specify the value for CONTAINER_LOG_FILES option,
+      // we will only output syslog
+      if (logFiles == null || logFiles.isEmpty()) {
+        logFiles = Arrays.asList("syslog");
       }
+      request.setLogTypes(logFiles);
       printContainerLogsFromRunningApplication(getConf(), request,
-          logFiles, logCliHelper, localDir);
+          logCliHelper);
     } 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(request,
-          requestedLogFiles, logCliHelper, localDir);
+      resultCode = printContainerLogsForFinishedApplication(
+          request, logCliHelper);
     }
     return resultCode;
   }
 
-  private int fetchApplicationLogs(ApplicationId appId, String appOwner,
-      LogCLIHelpers logCliHelper, String localDir) throws IOException {
-    int resultCode =
-        logCliHelper.dumpAllContainersLogs(appId, appOwner, localDir);
+  private int fetchApplicationLogs(ContainerLogsRequest options,
+      LogCLIHelpers logCliHelper) throws IOException {
+    // TODO: YARN-5141. To get container logs for the Running applications.
+    int resultCode = 0;
+    ContainerLogsRequest newOptions = getMatchedLogOptions(
+        options, logCliHelper);
+    if (newOptions == null) {
+      resultCode = -1;
+    } else {
+      resultCode =
+          logCliHelper.dumpAllContainersLogs(newOptions);
+    }
     if (resultCode == -1) {
       System.err.println("Can not find the logs for the application: "
-          + appId + " with the appOwner: " + appOwner);
+          + options.getAppId() + " with the appOwner: "
+          + options.getAppOwner());
     }
     return resultCode;
   }
@@ -833,80 +830,53 @@ public class LogsCLI extends Configured implements Tool {
     return appOwner;
   }
 
-  private static class ContainerLogsRequest {
-    private ApplicationId appId;
-    private String containerId;
-    private String nodeId;
-    private String nodeHttpAddress;
-    private String appOwner;
-    private boolean appFinished;
-
-    public ContainerLogsRequest(ContainerLogsRequest request) {
-      this.setAppId(request.getAppId());
-      this.setAppFinished(request.isAppFinished());
-      this.setAppOwner(request.getAppOwner());
-      this.setNodeId(request.getNodeId());
-      this.setNodeHttpAddress(request.getNodeHttpAddress());
-      this.setContainerId(request.getContainerId());
-    }
-
-    public ContainerLogsRequest(ApplicationId applicationId,
-        boolean isAppFinished, String owner,
-        String address, String httpAddress, String container) {
-      this.setAppId(applicationId);
-      this.setAppFinished(isAppFinished);
-      this.setAppOwner(owner);
-      this.setNodeId(address);
-      this.setNodeHttpAddress(httpAddress);
-      this.setContainerId(container);
-    }
-
-    public ApplicationId getAppId() {
-      return appId;
-    }
-
-    public void setAppId(ApplicationId appId) {
-      this.appId = appId;
-    }
-
-    public String getContainerId() {
-      return containerId;
-    }
-
-    public void setContainerId(String containerId) {
-      this.containerId = containerId;
-    }
-
-    public String getNodeId() {
-      return nodeId;
-    }
-
-    public void setNodeId(String nodeAddress) {
-      this.nodeId = nodeAddress;
-    }
-
-    public String getAppOwner() {
-      return appOwner;
-    }
-
-    public void setAppOwner(String appOwner) {
-      this.appOwner = appOwner;
-    }
-
-    public String getNodeHttpAddress() {
-      return nodeHttpAddress;
+  private ContainerLogsRequest getMatchedLogOptions(
+      ContainerLogsRequest request, LogCLIHelpers logCliHelper)
+      throws IOException {
+    ContainerLogsRequest newOptions = new ContainerLogsRequest(request);
+    if (request.getLogTypes() != null && !request.getLogTypes().isEmpty()) {
+      List<String> matchedFiles = new ArrayList<String>();
+      if (!request.getLogTypes().contains(".*")) {
+        Set<String> files = logCliHelper.listContainerLogs(request);
+        matchedFiles = getMatchedLogFiles(
+            request, files, true);
+        if (matchedFiles.isEmpty()) {
+          return null;
+        }
+      }
+      newOptions.setLogTypes(matchedFiles);
     }
+    return newOptions;
+  }
 
-    public void setNodeHttpAddress(String nodeHttpAddress) {
-      this.nodeHttpAddress = nodeHttpAddress;
+  private List<String> getMatchedLogFiles(ContainerLogsRequest options,
+      Collection<String> candidate, boolean printError) throws IOException {
+    List<String> matchedFiles = new ArrayList<String>();
+    List<String> filePattern = options.getLogTypes();
+    for (String file : candidate) {
+      if (isFileMatching(file, filePattern)) {
+        matchedFiles.add(file);
+      }
     }
-
-    public boolean isAppFinished() {
-      return appFinished;
+    if (matchedFiles.isEmpty()) {
+      if (printError) {
+        System.err.println("Can not find any log file matching the pattern: "
+            + options.getLogTypes() + " for the application: "
+            + options.getAppId());
+      }
     }
+    return matchedFiles;
+  }
 
-    public void setAppFinished(boolean appFinished) {
-      this.appFinished = appFinished;
+  private boolean isFileMatching(String fileType,
+      List<String> logTypes) {
+    for (String logType : logTypes) {
+      Pattern filterPattern = Pattern.compile(logType);
+      boolean match = filterPattern.matcher(fileType).find();
+      if (match) {
+        return true;
+      }
     }
+    return false;
   }
 }

+ 35 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java

@@ -181,7 +181,8 @@ public class TestLogsCLI {
     pw.println(" -logFiles <Log File Name>       Work with -am/-containerId and specify");
     pw.println("                                 comma-separated value to get specified");
     pw.println("                                 container log files. Use \"ALL\" to fetch");
-    pw.println("                                 all the log files for the container.");
+    pw.println("                                 all the log files for the container. It");
+    pw.println("                                 also supports Java Regex.");
     pw.println(" -nodeAddress <Node Address>     NodeAddress in the format nodename:port");
     pw.println(" -out <Local Directory>          Local directory for storing individual");
     pw.println("                                 container logs. The container logs will");
@@ -288,6 +289,39 @@ public class TestLogsCLI {
       "Hello container_0_0001_01_000003 in stdout!"));
     sysOutStream.reset();
 
+    exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
+        "-logFiles", ".*"});
+    assertTrue(exitCode == 0);
+    assertTrue(sysOutStream.toString().contains(
+        "Hello container_0_0001_01_000001 in syslog!"));
+    assertTrue(sysOutStream.toString().contains(
+        "Hello container_0_0001_01_000002 in syslog!"));
+    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();
+
+    exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
+        "-logFiles", "std*"});
+    assertTrue(exitCode == 0);
+    assertFalse(sysOutStream.toString().contains(
+        "Hello container_0_0001_01_000001 in syslog!"));
+    assertFalse(sysOutStream.toString().contains(
+        "Hello container_0_0001_01_000002 in syslog!"));
+    assertFalse(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();
+
+    exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
+        "-logFiles", "123"});
+    assertTrue(exitCode == -1);
+    assertTrue(sysErrStream.toString().contains(
+        "Can not find any log file matching the pattern: [123]"));
+    sysErrStream.reset();
+
     // uploaded two logs for container1. The first log is empty.
     // The second one is not empty.
     // We can still successfully read logs for container1.

+ 8 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java

@@ -875,23 +875,25 @@ public class AggregatedLogFormat {
     }
 
     @Private
-    public static void readContainerMetaDataAndSkipData(
+    public static String readContainerMetaDataAndSkipData(
         DataInputStream valueStream, PrintStream out) throws IOException {
 
       String fileType = valueStream.readUTF();
       String fileLengthStr = valueStream.readUTF();
       long fileLength = Long.parseLong(fileLengthStr);
-      out.print("LogType:");
-      out.println(fileType);
-      out.print("LogLength:");
-      out.println(fileLengthStr);
-
+      if (out != null) {
+        out.print("LogType:");
+        out.println(fileType);
+        out.print("LogLength:");
+        out.println(fileLengthStr);
+      }
       long totalSkipped = 0;
       long currSkipped = 0;
       while (currSkipped != -1 && totalSkipped < fileLength) {
         currSkipped = valueStream.skip(fileLength - totalSkipped);
         totalSkipped += currSkipped;
       }
+      return fileType;
     }
 
     public void close() {

+ 124 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogsRequest.java

@@ -0,0 +1,124 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.logaggregation;
+
+import java.util.List;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+
+public class ContainerLogsRequest {
+  private ApplicationId appId;
+  private String containerId;
+  private String nodeId;
+  private String nodeHttpAddress;
+  private String appOwner;
+  private boolean appFinished;
+  private String outputLocalDir;
+  private List<String> logTypes;
+
+  public ContainerLogsRequest() {}
+
+  public ContainerLogsRequest(ContainerLogsRequest request) {
+    this.setAppId(request.getAppId());
+    this.setAppFinished(request.isAppFinished());
+    this.setAppOwner(request.getAppOwner());
+    this.setNodeId(request.getNodeId());
+    this.setNodeHttpAddress(request.getNodeHttpAddress());
+    this.setContainerId(request.getContainerId());
+    this.setOutputLocalDir(request.getOutputLocalDir());
+    this.setLogTypes(request.getLogTypes());
+  }
+
+  public ContainerLogsRequest(ApplicationId applicationId,
+      boolean isAppFinished, String owner,
+      String address, String httpAddress, String container, String localDir,
+      List<String> logs) {
+    this.setAppId(applicationId);
+    this.setAppFinished(isAppFinished);
+    this.setAppOwner(owner);
+    this.setNodeId(address);
+    this.setNodeHttpAddress(httpAddress);
+    this.setContainerId(container);
+    this.setOutputLocalDir(localDir);
+    this.setLogTypes(logs);
+  }
+
+  public ApplicationId getAppId() {
+    return appId;
+  }
+
+  public void setAppId(ApplicationId appId) {
+    this.appId = appId;
+  }
+
+  public String getContainerId() {
+    return containerId;
+  }
+
+  public void setContainerId(String containerId) {
+    this.containerId = containerId;
+  }
+
+  public String getNodeId() {
+    return nodeId;
+  }
+
+  public void setNodeId(String nodeAddress) {
+    this.nodeId = nodeAddress;
+  }
+
+  public String getAppOwner() {
+    return appOwner;
+  }
+
+  public void setAppOwner(String appOwner) {
+    this.appOwner = appOwner;
+  }
+
+  public String getNodeHttpAddress() {
+    return nodeHttpAddress;
+  }
+
+  public void setNodeHttpAddress(String nodeHttpAddress) {
+    this.nodeHttpAddress = nodeHttpAddress;
+  }
+
+  public boolean isAppFinished() {
+    return appFinished;
+  }
+
+  public void setAppFinished(boolean appFinished) {
+    this.appFinished = appFinished;
+  }
+
+  public String getOutputLocalDir() {
+    return outputLocalDir;
+  }
+
+  public void setOutputLocalDir(String outputLocalDir) {
+    this.outputLocalDir = outputLocalDir;
+  }
+
+  public List<String> getLogTypes() {
+    return logTypes;
+  }
+
+  public void setLogTypes(List<String> logTypes) {
+    this.logTypes = logTypes;
+  }
+}

+ 115 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java

@@ -26,7 +26,10 @@ import java.io.PrintStream;
 import java.nio.file.AccessDeniedException;
 import java.nio.file.Files;
 import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -55,8 +58,14 @@ public class LogCLIHelpers implements Configurable {
   @VisibleForTesting
   public int dumpAContainersLogs(String appId, String containerId,
       String nodeId, String jobOwner) throws IOException {
-    return dumpAContainersLogsForALogType(appId, containerId, nodeId, jobOwner,
-      null, null);
+    ContainerLogsRequest options = new ContainerLogsRequest();
+    options.setAppId(ConverterUtils.toApplicationId(appId));
+    options.setContainerId(containerId);
+    options.setNodeId(nodeId);
+    options.setAppOwner(jobOwner);
+    List<String> logs = new ArrayList<String>();
+    options.setLogTypes(logs);
+    return dumpAContainersLogsForALogType(options, false);
   }
 
   @Private
@@ -108,19 +117,21 @@ public class LogCLIHelpers implements Configurable {
 
   @Private
   @VisibleForTesting
-  public int dumpAContainersLogsForALogType(String appId, String containerId,
-      String nodeId, String jobOwner, List<String> logType, String localDir)
+  public int dumpAContainersLogsForALogType(ContainerLogsRequest options)
       throws IOException {
-    return dumpAContainersLogsForALogType(appId, containerId, nodeId,
-        jobOwner, logType, true, localDir);
+    return dumpAContainersLogsForALogType(options, true);
   }
 
   @Private
   @VisibleForTesting
-  public int dumpAContainersLogsForALogType(String appId, String containerId,
-      String nodeId, String jobOwner, List<String> logType,
-      boolean outputFailure, String localDir) throws IOException {
-    ApplicationId applicationId = ConverterUtils.toApplicationId(appId);
+  public int dumpAContainersLogsForALogType(ContainerLogsRequest options,
+      boolean outputFailure) throws IOException {
+    ApplicationId applicationId = options.getAppId();
+    String jobOwner = options.getAppOwner();
+    String nodeId = options.getNodeId();
+    String containerId = options.getContainerId();
+    String localDir = options.getOutputLocalDir();
+    List<String> logType = options.getLogTypes();
     RemoteIterator<FileStatus> nodeFiles = getRemoteNodeFileDir(
         applicationId, jobOwner);
     if (nodeFiles == null) {
@@ -147,7 +158,7 @@ public class LogCLIHelpers implements Configurable {
           reader =
               new AggregatedLogFormat.LogReader(getConf(),
                 thisNodeFile.getPath());
-          if (logType == null) {
+          if (logType == null || logType.isEmpty()) {
             if (dumpAContainerLogs(containerId, reader, out,
                 thisNodeFile.getModificationTime()) > -1) {
               foundContainerLogs = true;
@@ -176,10 +187,13 @@ public class LogCLIHelpers implements Configurable {
   }
 
   @Private
-  public int dumpAContainersLogsForALogTypeWithoutNodeId(String appId,
-      String containerId, String jobOwner, List<String> logType,
-      String localDir) throws IOException {
-    ApplicationId applicationId = ConverterUtils.toApplicationId(appId);
+  public int dumpAContainersLogsForALogTypeWithoutNodeId(
+      ContainerLogsRequest options) throws IOException {
+    ApplicationId applicationId = options.getAppId();
+    String jobOwner = options.getAppOwner();
+    String containerId = options.getContainerId();
+    String localDir = options.getOutputLocalDir();
+    List<String> logType = options.getLogTypes();
     RemoteIterator<FileStatus> nodeFiles = getRemoteNodeFileDir(
         applicationId, jobOwner);
     if (nodeFiles == null) {
@@ -206,7 +220,7 @@ public class LogCLIHelpers implements Configurable {
               containerId);
           out.println(containerId);
           out.println(StringUtils.repeat("=", containerId.length()));
-          if (logType == null) {
+          if (logType == null || logType.isEmpty()) {
             if (dumpAContainerLogs(containerId, reader, out,
                 thisNodeFile.getModificationTime()) > -1) {
               foundContainerLogs = true;
@@ -303,8 +317,12 @@ public class LogCLIHelpers implements Configurable {
   }
 
   @Private
-  public int dumpAllContainersLogs(ApplicationId appId, String appOwner,
-      String localDir) throws IOException {
+  public int dumpAllContainersLogs(ContainerLogsRequest options)
+      throws IOException {
+    ApplicationId appId = options.getAppId();
+    String appOwner = options.getAppOwner();
+    String localDir = options.getOutputLocalDir();
+    List<String> logTypes = options.getLogTypes();
     RemoteIterator<FileStatus> nodeFiles = getRemoteNodeFileDir(
         appId, appOwner);
     if (nodeFiles == null) {
@@ -341,9 +359,18 @@ public class LogCLIHelpers implements Configurable {
               out.println(StringUtils.repeat("=", containerString.length()));
               while (true) {
                 try {
-                  LogReader.readAContainerLogsForALogType(valueStream, out,
-                      thisNodeFile.getModificationTime());
-                  foundAnyLogs = true;
+                  if (logTypes == null || logTypes.isEmpty()) {
+                    LogReader.readAContainerLogsForALogType(valueStream, out,
+                        thisNodeFile.getModificationTime());
+                    foundAnyLogs = true;
+                  } else {
+                    int result = LogReader.readContainerLogsForALogType(
+                        valueStream, out, thisNodeFile.getModificationTime(),
+                        logTypes);
+                    if (result == 0) {
+                      foundAnyLogs = true;
+                    }
+                  }
                 } catch (EOFException eof) {
                   break;
                 }
@@ -369,10 +396,13 @@ public class LogCLIHelpers implements Configurable {
   }
 
   @Private
-  public void printLogMetadata(ApplicationId appId,
-      String containerIdStr, String nodeId, String appOwner,
+  public void printLogMetadata(ContainerLogsRequest options,
       PrintStream out, PrintStream err)
       throws IOException {
+    ApplicationId appId = options.getAppId();
+    String appOwner = options.getAppOwner();
+    String nodeId = options.getNodeId();
+    String containerIdStr = options.getContainerId();
     boolean getAllContainers = (containerIdStr == null);
     String nodeIdStr = (nodeId == null) ? null
         : LogAggregationUtils.getNodeString(nodeId);
@@ -443,8 +473,10 @@ public class LogCLIHelpers implements Configurable {
   }
 
   @Private
-  public void printNodesList(ApplicationId appId, String appOwner,
+  public void printNodesList(ContainerLogsRequest options,
       PrintStream out, PrintStream err) throws IOException {
+    ApplicationId appId = options.getAppId();
+    String appOwner = options.getAppOwner();
     RemoteIterator<FileStatus> nodeFiles = getRemoteNodeFileDir(
         appId, appOwner);
     if (nodeFiles == null) {
@@ -547,4 +579,63 @@ public class LogCLIHelpers implements Configurable {
       IOUtils.closeQuietly(out);
     }
   }
+
+  @Private
+  public Set<String> listContainerLogs(ContainerLogsRequest options)
+      throws IOException {
+    Set<String> logTypes = new HashSet<String>();
+    ApplicationId appId = options.getAppId();
+    String appOwner = options.getAppOwner();
+    String nodeId = options.getNodeId();
+    String containerIdStr = options.getContainerId();
+    boolean getAllContainers = (containerIdStr == null);
+    String nodeIdStr = (nodeId == null) ? null
+        : LogAggregationUtils.getNodeString(nodeId);
+    RemoteIterator<FileStatus> nodeFiles = getRemoteNodeFileDir(
+        appId, appOwner);
+    if (nodeFiles == null) {
+      return logTypes;
+    }
+    while (nodeFiles.hasNext()) {
+      FileStatus thisNodeFile = nodeFiles.next();
+      if (nodeIdStr != null) {
+        if (!thisNodeFile.getPath().getName().contains(nodeIdStr)) {
+          continue;
+        }
+      }
+      if (!thisNodeFile.getPath().getName()
+          .endsWith(LogAggregationUtils.TMP_FILE_SUFFIX)) {
+        AggregatedLogFormat.LogReader reader =
+            new AggregatedLogFormat.LogReader(getConf(),
+            thisNodeFile.getPath());
+        try {
+          DataInputStream valueStream;
+          LogKey key = new LogKey();
+          valueStream = reader.next(key);
+          while (valueStream != null) {
+            if (getAllContainers || (key.toString().equals(containerIdStr))) {
+              while (true) {
+                try {
+                  String logFile = LogReader.readContainerMetaDataAndSkipData(
+                      valueStream, null);
+                  logTypes.add(logFile);
+                } catch (EOFException eof) {
+                  break;
+                }
+              }
+              if (!getAllContainers) {
+                break;
+              }
+            }
+            // Next container
+            key = new LogKey();
+            valueStream = reader.next(key);
+          }
+        } finally {
+          reader.close();
+        }
+      }
+    }
+    return logTypes;
+  }
 }