Forráskód Böngészése

YARN-5363. For AM containers, or for containers of running-apps, "yarn logs" incorrectly only (tries to) shows syslog file-type by default. (Contributed by Vinod Kumar Vavilapalli)

Xuan 8 éve
szülő
commit
429347289c

+ 25 - 38
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java

@@ -28,7 +28,9 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 import java.util.regex.Pattern;
+
 import javax.ws.rs.core.MediaType;
+
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -53,9 +55,8 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 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.logaggregation.LogCLIHelpers;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -78,7 +79,7 @@ public class LogsCLI extends Configured implements Tool {
   private static final String NODE_ADDRESS_OPTION = "nodeAddress";
   private static final String APP_OWNER_OPTION = "appOwner";
   private static final String AM_CONTAINER_OPTION = "am";
-  private static final String CONTAINER_LOG_FILES = "logFiles";
+  private static final String PER_CONTAINER_LOG_FILES_OPTION = "log_files";
   private static final String LIST_NODES_OPTION = "list_nodes";
   private static final String SHOW_APPLICATION_LOG_INFO
       = "show_application_log_info";
@@ -146,8 +147,8 @@ public class LogsCLI extends Configured implements Tool {
           return -1;
         }
       }
-      if (commandLine.hasOption(CONTAINER_LOG_FILES)) {
-        logFiles = commandLine.getOptionValues(CONTAINER_LOG_FILES);
+      if (commandLine.hasOption(PER_CONTAINER_LOG_FILES_OPTION)) {
+        logFiles = commandLine.getOptionValues(PER_CONTAINER_LOG_FILES_OPTION);
       }
       if (commandLine.hasOption(SIZE_OPTION)) {
         bytes = Long.parseLong(commandLine.getOptionValue(SIZE_OPTION));
@@ -359,12 +360,18 @@ 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") || logs.contains(".*")) {
-        return true;
-      }
+
+    // If no value is specified for the PER_CONTAINER_LOG_FILES_OPTION option,
+    // we will assume all logs.
+    if (logFiles == null || logFiles.length == 0) {
+      return true;
     }
+
+    List<String> logs = Arrays.asList(logFiles);
+    if (logs.contains("ALL") || logs.contains(".*")) {
+      return true;
+    }
+
     return false;
   }
 
@@ -424,7 +431,7 @@ public class LogsCLI extends Configured implements Tool {
         containerIdStr);
     try {
       // fetch all the log files for the container
-      // filter the log files based on the given --logFiles pattern
+      // filter the log files based on the given -log_files pattern
       List<PerLogFileInfo> allLogFileInfos=
           getContainerLogFiles(getConf(), containerIdStr, nodeHttpAddress);
       List<String> fileNames = new ArrayList<String>();
@@ -700,8 +707,8 @@ public class LogsCLI extends Configured implements Tool {
         new Option(APPLICATION_ID_OPTION, true, "ApplicationId (required)");
     opts.addOption(appIdOpt);
     opts.addOption(CONTAINER_ID_OPTION, true, "ContainerId. "
-        + "By default, it will only print syslog if the application is running."
-        + " Work with -logFiles to get other logs. If specified, the"
+        + "By default, it will print all available logs."
+        + " Work with -log_files to get only specific logs. If specified, the"
         + " applicationId can be omitted");
     opts.addOption(NODE_ADDRESS_OPTION, true, "NodeAddress in the format "
         + "nodename:port");
@@ -714,13 +721,13 @@ public class LogsCLI extends Configured implements Tool {
         + "the logs for the first AM Container as well as the second "
         + "AM Container. To get logs for all AM Containers, use -am ALL. "
         + "To get logs for the latest AM Container, use -am -1. "
-        + "By default, it will only print out syslog. Work with -logFiles "
-        + "to get other logs");
+        + "By default, it will print all available logs. Work with -log_files "
+        + "to get only specific logs.");
     amOption.setValueSeparator(',');
     amOption.setArgs(Option.UNLIMITED_VALUES);
     amOption.setArgName("AM Containers");
     opts.addOption(amOption);
-    Option logFileOpt = new Option(CONTAINER_LOG_FILES, true,
+    Option logFileOpt = new Option(PER_CONTAINER_LOG_FILES_OPTION, true,
         "Specify comma-separated value "
         + "to get specified container log files. Use \"ALL\" to fetch all the "
         + "log files for the container. It also supports Java Regex.");
@@ -764,7 +771,7 @@ public class LogsCLI extends Configured implements Tool {
     printOpts.addOption(commandOpts.getOption(NODE_ADDRESS_OPTION));
     printOpts.addOption(commandOpts.getOption(APP_OWNER_OPTION));
     printOpts.addOption(commandOpts.getOption(AM_CONTAINER_OPTION));
-    printOpts.addOption(commandOpts.getOption(CONTAINER_LOG_FILES));
+    printOpts.addOption(commandOpts.getOption(PER_CONTAINER_LOG_FILES_OPTION));
     printOpts.addOption(commandOpts.getOption(LIST_NODES_OPTION));
     printOpts.addOption(commandOpts.getOption(SHOW_APPLICATION_LOG_INFO));
     printOpts.addOption(commandOpts.getOption(SHOW_CONTAINER_LOG_INFO));
@@ -807,13 +814,7 @@ public class LogsCLI extends Configured implements Tool {
   private int fetchAMContainerLogs(ContainerLogsRequest request,
       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.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.
@@ -856,7 +857,6 @@ public class LogsCLI extends Configured implements Tool {
     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) {
@@ -899,12 +899,6 @@ public class LogsCLI extends Configured implements Tool {
     // we will provide the NodeHttpAddress and get the container logs
     // by calling NodeManager webservice.
     if (!isAppFinished) {
-      // 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);
       resultCode = printContainerLogsFromRunningApplication(getConf(), request,
           logCliHelper);
     } else {
@@ -1026,13 +1020,6 @@ public class LogsCLI extends Configured implements Tool {
         newOptions.setNodeHttpAddress(httpAddress
             .replaceFirst(WebAppUtils.getHttpSchemePrefix(getConf()), ""));
       }
-      // if we do not specify the value for CONTAINER_LOG_FILES option,
-      // we will only output syslog
-      List<String> logFiles = newOptions.getLogTypes();
-      if (logFiles == null || logFiles.isEmpty()) {
-        logFiles = Arrays.asList("syslog");
-        newOptions.setLogTypes(logFiles);
-      }
       newOptionsList.add(newOptions);
     }
     return newOptionsList;

+ 39 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java

@@ -46,6 +46,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -74,6 +75,7 @@ import org.codehaus.jettison.json.JSONObject;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.ArgumentCaptor;
 
 public class TestLogsCLI {
 
@@ -195,20 +197,20 @@ public class TestLogsCLI {
     pw.println("                                 Container. To get logs for all AM");
     pw.println("                                 Containers, use -am ALL. To get logs for");
     pw.println("                                 the latest AM Container, use -am -1. By");
-    pw.println("                                 default, it will only print out syslog.");
-    pw.println("                                 Work with -logFiles to get other logs");
+    pw.println("                                 default, it will print all available");
+    pw.println("                                 logs. Work with -log_files to get only");
+    pw.println("                                 specific logs.");
     pw.println(" -appOwner <Application Owner>   AppOwner (assumed to be current user if");
     pw.println("                                 not specified)");
-    pw.println(" -containerId <Container ID>     ContainerId. By default, it will only");
-    pw.println("                                 print syslog if the application is");
-    pw.println("                                 running. Work with -logFiles to get other");
-    pw.println("                                 logs. If specified, the applicationId can");
-    pw.println("                                 be omitted");
+    pw.println(" -containerId <Container ID>     ContainerId. By default, it will print");
+    pw.println("                                 all available logs. Work with -log_files");
+    pw.println("                                 to get only specific logs. If specified,");
+    pw.println("                                 the applicationId can be omitted");
     pw.println(" -help                           Displays help for all commands.");
     pw.println(" -list_nodes                     Show the list of nodes that successfully");
     pw.println("                                 aggregated logs. This option can only be");
     pw.println("                                 used with finished applications.");
-    pw.println(" -logFiles <Log File Name>       Specify comma-separated value to get");
+    pw.println(" -log_files <Log File Name>      Specify comma-separated value to get");
     pw.println("                                 specified container log files. Use \"ALL\"");
     pw.println("                                 to fetch all the log files for the");
     pw.println("                                 container. It also supports Java Regex.");
@@ -240,7 +242,7 @@ public class TestLogsCLI {
   }
 
   @Test (timeout = 15000)
-  public void testFetchApplictionLogs() throws Exception {
+  public void testFetchFinishedApplictionLogs() throws Exception {
     String remoteLogRootDir = "target/logs/";
     Configuration configuration = new Configuration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
@@ -337,7 +339,7 @@ public class TestLogsCLI {
     sysOutStream.reset();
 
     exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
-        "-logFiles", ".*"});
+        "-log_files", ".*"});
     assertTrue(exitCode == 0);
     assertTrue(sysOutStream.toString().contains(
         "Hello container_0_0001_01_000001 in syslog!"));
@@ -351,7 +353,7 @@ public class TestLogsCLI {
     sysOutStream.reset();
 
     exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
-        "-logFiles", "std*"});
+        "-log_files", "std*"});
     assertTrue(exitCode == 0);
     assertFalse(sysOutStream.toString().contains(
         "Hello container_0_0001_01_000001 in syslog!"));
@@ -364,7 +366,7 @@ public class TestLogsCLI {
     sysOutStream.reset();
 
     exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
-        "-logFiles", "123"});
+        "-log_files", "123"});
     assertTrue(exitCode == -1);
     assertTrue(sysErrStream.toString().contains(
         "Can not find any log file matching the pattern: [123] "
@@ -374,7 +376,7 @@ public class TestLogsCLI {
     // specify the bytes which is larger than the actual file size,
     // we would get the full logs
     exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
-        "-logFiles", ".*", "-size", "10000" });
+        "-log_files", ".*", "-size", "10000" });
     assertTrue(exitCode == 0);
     assertTrue(sysOutStream.toByteArray().length == fullSize);
     sysOutStream.reset();
@@ -396,7 +398,7 @@ public class TestLogsCLI {
     sysOutStream.reset();
 
     exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
-        "-containerId", containerId3.toString(), "-logFiles", "123" });
+        "-containerId", containerId3.toString(), "-log_files", "123" });
     assertTrue(exitCode == -1);
     assertTrue(sysErrStream.toString().contains(
         "Can not find any log file matching the pattern: [123] "
@@ -405,7 +407,7 @@ public class TestLogsCLI {
     sysErrStream.reset();
 
     exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
-        "-containerId", containerId3.toString(), "-logFiles", "stdout" });
+        "-containerId", containerId3.toString(), "-log_files", "stdout" });
     assertTrue(exitCode == 0);
     int fullContextSize = sysOutStream.toByteArray().length;
     String fullContext = sysOutStream.toString();
@@ -419,7 +421,7 @@ public class TestLogsCLI {
     // specify a position number, it would get the first n bytes from
     // container log
     exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
-        "-containerId", containerId3.toString(), "-logFiles", "stdout",
+        "-containerId", containerId3.toString(), "-log_files", "stdout",
         "-size", "5"});
     assertTrue(exitCode == 0);
     Assert.assertEquals(new String(logMessage.getBytes(), 0, 5),
@@ -430,7 +432,7 @@ public class TestLogsCLI {
     // specify a negative number, it would get the last n bytes from
     // container log
     exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
-        "-containerId", containerId3.toString(), "-logFiles", "stdout",
+        "-containerId", containerId3.toString(), "-log_files", "stdout",
         "-size", "-5"});
     assertTrue(exitCode == 0);
     Assert.assertEquals(new String(logMessage.getBytes(),
@@ -441,7 +443,7 @@ public class TestLogsCLI {
 
     long negative = (fullContextSize + 1000) * (-1);
     exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
-        "-containerId", containerId3.toString(), "-logFiles", "stdout",
+        "-containerId", containerId3.toString(), "-log_files", "stdout",
         "-size", Long.toString(negative)});
     assertTrue(exitCode == 0);
     Assert.assertEquals(fullContext, sysOutStream.toString());
@@ -473,12 +475,12 @@ public class TestLogsCLI {
         "Hello container_0_0001_01_000003 in stdout!"));
     sysOutStream.reset();
 
-    // set -logFiles option as stdout
+    // set -log_files option as stdout
     // should only print log with the name as stdout
     exitCode =
         cli.run(new String[] { "-applicationId", appId.toString(),
             "-nodeAddress", nodeId.toString(), "-containerId",
-            containerId3.toString() , "-logFiles", "stdout"});
+            containerId3.toString() , "-log_files", "stdout"});
     assertTrue(exitCode == 0);
     assertTrue(sysOutStream.toString().contains(
         "Hello container_0_0001_01_000003 in stdout!"));
@@ -527,11 +529,14 @@ public class TestLogsCLI {
 
   @Test (timeout = 5000)
   public void testFetchRunningApplicationLogs() throws Exception {
+
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+
     NodeId nodeId = NodeId.newInstance("localhost", 1234);
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     ApplicationAttemptId appAttemptId = ApplicationAttemptId
         .newInstance(appId, 1);
+
     // Create a mock ApplicationAttempt Report
     ApplicationAttemptReport mockAttemptReport = mock(
         ApplicationAttemptReport.class);
@@ -568,11 +573,24 @@ public class TestLogsCLI {
     cli.setConf(new YarnConfiguration());
     int exitCode = cli.run(new String[] {"-applicationId", appId.toString()});
     assertTrue(exitCode == 0);
+
+    ArgumentCaptor<ContainerLogsRequest> logsRequestCaptor =
+        ArgumentCaptor.forClass(ContainerLogsRequest.class);
+
     // we have two container reports, so make sure we have called
     // printContainerLogsFromRunningApplication twice
     verify(cli, times(2)).printContainerLogsFromRunningApplication(
-        any(Configuration.class), any(ContainerLogsRequest.class),
+        any(Configuration.class), logsRequestCaptor.capture(),
         any(LogCLIHelpers.class));
+
+    // Verify that the log-type is *
+    List<ContainerLogsRequest> capturedRequests =
+        logsRequestCaptor.getAllValues();
+    Assert.assertEquals(2, capturedRequests.size());
+    List<String> logTypes0 = capturedRequests.get(0).getLogTypes();
+    List<String> logTypes1 = capturedRequests.get(1).getLogTypes();
+    Assert.assertEquals(".*", logTypes0.get(0));
+    Assert.assertEquals(".*", logTypes1.get(0));
   }
 
   @Test (timeout = 15000)