Selaa lähdekoodia

YARN-10106. Yarn logs CLI filtering by application attempt. Contributed by Hudáky Márton Gyula

Adam Antal 4 vuotta sitten
vanhempi
commit
c049296235

+ 36 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java

@@ -65,6 +65,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -95,6 +96,8 @@ public class LogsCLI extends Configured implements Tool {
 
   private static final String CONTAINER_ID_OPTION = "containerId";
   private static final String APPLICATION_ID_OPTION = "applicationId";
+  private static final String APPLICATION_ATTEMPT_ID_OPTION =
+          "applicationAttemptId";
   private static final String CLUSTER_ID_OPTION = "clusterId";
   private static final String NODE_ADDRESS_OPTION = "nodeAddress";
   private static final String APP_OWNER_OPTION = "appOwner";
@@ -160,6 +163,7 @@ public class LogsCLI extends Configured implements Tool {
     }
     CommandLineParser parser = new GnuParser();
     String appIdStr = null;
+    String appAttemptIdStr = null;
     String clusterIdStr = null;
     String containerIdStr = null;
     String nodeAddress = null;
@@ -180,6 +184,8 @@ public class LogsCLI extends Configured implements Tool {
     try {
       CommandLine commandLine = parser.parse(opts, args, false);
       appIdStr = commandLine.getOptionValue(APPLICATION_ID_OPTION);
+      appAttemptIdStr = commandLine.getOptionValue(
+              APPLICATION_ATTEMPT_ID_OPTION);
       containerIdStr = commandLine.getOptionValue(CONTAINER_ID_OPTION);
       nodeAddress = commandLine.getOptionValue(NODE_ADDRESS_OPTION);
       appOwner = commandLine.getOptionValue(APP_OWNER_OPTION);
@@ -240,9 +246,9 @@ public class LogsCLI extends Configured implements Tool {
       return -1;
     }
 
-    if (appIdStr == null && containerIdStr == null) {
-      System.err.println("Both applicationId and containerId are missing, "
-          + " one of them must be specified.");
+    if (appIdStr == null && appAttemptIdStr == null && containerIdStr == null) {
+      System.err.println("None of applicationId, appAttemptId and containerId "
+          + "is available,  one of them must be specified.");
       printHelpMessage(printOpts);
       return -1;
     }
@@ -257,9 +263,32 @@ public class LogsCLI extends Configured implements Tool {
       }
     }
 
+    ApplicationAttemptId appAttemptId = null;
+    if (appAttemptIdStr != null) {
+      try {
+        appAttemptId = ApplicationAttemptId.fromString(appAttemptIdStr);
+        if (appId == null) {
+          appId = appAttemptId.getApplicationId();
+        } else if (!appId.equals(appAttemptId.getApplicationId())) {
+          System.err.println("The Application:" + appId
+                  + " does not have the AppAttempt:" + appAttemptId);
+          return -1;
+        }
+      } catch (Exception e) {
+        System.err.println("Invalid AppAttemptId specified");
+        return -1;
+      }
+    }
+
     if (containerIdStr != null) {
       try {
         ContainerId containerId = ContainerId.fromString(containerIdStr);
+        if (appAttemptId != null && !appAttemptId.equals(
+                containerId.getApplicationAttemptId())) {
+          System.err.println("The AppAttempt:" + appAttemptId
+                  + " does not have the container:" + containerId);
+          return -1;
+        }
         if (appId == null) {
           appId = containerId.getApplicationAttemptId().getApplicationId();
         } else if (!containerId.getApplicationAttemptId().getApplicationId()
@@ -344,7 +373,7 @@ public class LogsCLI extends Configured implements Tool {
     }
 
 
-    ContainerLogsRequest request = new ContainerLogsRequest(appId, null,
+    ContainerLogsRequest request = new ContainerLogsRequest(appId, appAttemptId,
         Apps.isApplicationFinalState(appState), appOwner, nodeAddress,
         null, containerIdStr, localDir, logs, bytes, null);
 
@@ -913,6 +942,9 @@ public class LogsCLI extends Configured implements Tool {
     Option appIdOpt =
         new Option(APPLICATION_ID_OPTION, true, "ApplicationId (required)");
     opts.addOption(appIdOpt);
+    opts.addOption(APPLICATION_ATTEMPT_ID_OPTION, true, "ApplicationAttemptId. "
+        + "Lists all logs belonging to the specified application attempt Id. "
+        + "If specified, the applicationId can be omitted");
     opts.addOption(CONTAINER_ID_OPTION, true, "ContainerId. "
         + "By default, it will print all available logs."
         + " Work with -log_files to get only specific logs. If specified, the"

+ 156 - 29
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java

@@ -238,6 +238,22 @@ public class TestLogsCLI {
         "Unable to get ApplicationState"));
   }
 
+  @Test(timeout = 5000L)
+  public void testUnknownApplicationAttemptId() throws Exception {
+    YarnClient mockYarnClient = createMockYarnClientUnknownApp();
+    LogsCLI cli = new LogsCLIForTest(mockYarnClient);
+    cli.setConf(conf);
+    ApplicationId appId = ApplicationId.newInstance(0, 1);
+
+    int exitCode = cli.run(new String[] {"-applicationAttemptId",
+            ApplicationAttemptId.newInstance(appId, 1).toString() });
+
+    // Error since no logs present for the app.
+    assertTrue(exitCode != 0);
+    assertTrue(sysErrStream.toString().contains(
+            "Unable to get ApplicationState."));
+  }
+
   @Test (timeout = 10000)
   public void testHelpMessage() throws Exception {
     YarnClient mockYarnClient = createMockYarnClient(
@@ -372,12 +388,14 @@ public class TestLogsCLI {
 
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     ApplicationId appId = ApplicationId.newInstance(0, 1);
-    ApplicationAttemptId appAttemptId =
+    ApplicationAttemptId appAttemptId1 =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId0 = ContainerId.newContainerId(appAttemptId, 0);
-    ContainerId containerId1 = ContainerId.newContainerId(appAttemptId, 1);
-    ContainerId containerId2 = ContainerId.newContainerId(appAttemptId, 2);
-    ContainerId containerId3 = ContainerId.newContainerId(appAttemptId, 3);
+    ApplicationAttemptId appAttemptId2 =
+            ApplicationAttemptId.newInstance(appId, 2);
+    ContainerId containerId0 = ContainerId.newContainerId(appAttemptId1, 0);
+    ContainerId containerId1 = ContainerId.newContainerId(appAttemptId1, 1);
+    ContainerId containerId2 = ContainerId.newContainerId(appAttemptId1, 2);
+    ContainerId containerId3 = ContainerId.newContainerId(appAttemptId2, 3);
     final NodeId nodeId = NodeId.newInstance("localhost", 1234);
 
     // create local logs
@@ -464,6 +482,44 @@ public class TestLogsCLI {
         createEmptyLog("empty")));
     sysOutStream.reset();
 
+    // Check fetching data for application attempt with applicationId defined
+    exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
+        "-applicationAttemptId", appAttemptId1.toString()});
+    LOG.info(sysOutStream.toString());
+    assertTrue(exitCode == 0);
+    assertTrue(sysOutStream.toString().contains(
+            logMessage(containerId1, "syslog")));
+    assertTrue(sysOutStream.toString().contains(
+            logMessage(containerId2, "syslog")));
+    assertTrue(sysOutStream.toString().contains(
+            logMessage(containerId3, "syslog")));
+    assertFalse(sysOutStream.toString().contains(
+            logMessage(containerId3, "stdout")));
+    assertFalse(sysOutStream.toString().contains(
+            logMessage(containerId3, "stdout1234")));
+    assertTrue(sysOutStream.toString().contains(
+            createEmptyLog("empty")));
+    sysOutStream.reset();
+
+    // Check fetching data for application attempt without application defined
+    exitCode = cli.run(new String[] {
+        "-applicationAttemptId", appAttemptId1.toString()});
+    LOG.info(sysOutStream.toString());
+    assertTrue(exitCode == 0);
+    assertTrue(sysOutStream.toString().contains(
+            logMessage(containerId1, "syslog")));
+    assertTrue(sysOutStream.toString().contains(
+            logMessage(containerId2, "syslog")));
+    assertTrue(sysOutStream.toString().contains(
+            logMessage(containerId3, "syslog")));
+    assertFalse(sysOutStream.toString().contains(
+            logMessage(containerId3, "stdout")));
+    assertFalse(sysOutStream.toString().contains(
+            logMessage(containerId3, "stdout1234")));
+    assertTrue(sysOutStream.toString().contains(
+            createEmptyLog("empty")));
+    sysOutStream.reset();
+
     exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
         "-log_files_pattern", ".*"});
     assertTrue(exitCode == 0);
@@ -979,6 +1035,8 @@ public class TestLogsCLI {
         any(ContainerLogsRequest.class));
     cli2.setConf(new YarnConfiguration());
     ContainerId containerId100 = ContainerId.newContainerId(appAttemptId, 100);
+    System.out.println(containerId100.toString());
+    System.out.println(appId.toString());
     exitCode = cli2.run(new String[] {"-applicationId", appId.toString(),
         "-containerId", containerId100.toString(), "-nodeAddress", "NM:1234"});
     assertTrue(exitCode == 0);
@@ -1139,52 +1197,112 @@ public class TestLogsCLI {
   }
 
   @Test (timeout = 5000)
-  public void testLogsCLIWithInvalidArgs() throws Exception {
-    String localDir = "target/SaveLogs";
-    Path localPath = new Path(localDir);
-    FileSystem fs = FileSystem.get(conf);
-    ApplicationId appId = ApplicationId.newInstance(0, 1);
-    YarnClient mockYarnClient =
-        createMockYarnClient(YarnApplicationState.FINISHED,
-        UserGroupInformation.getCurrentUser().getShortUserName());
-    LogsCLI cli = new LogsCLIForTest(mockYarnClient);
-    cli.setConf(conf);
+  public void testWithInvalidApplicationId() throws Exception {
+    LogsCLI cli = createCli();
 
     // Specify an invalid applicationId
-    int exitCode = cli.run(new String[] {"-applicationId",
-        "123"});
+    int exitCode = cli.run(new String[] {"-applicationId", "123"});
     assertTrue(exitCode == -1);
     assertTrue(sysErrStream.toString().contains(
-        "Invalid ApplicationId specified"));
+            "Invalid ApplicationId specified"));
+  }
+
+  @Test (timeout = 5000)
+  public void testWithInvalidAppAttemptId() throws Exception {
+    LogsCLI cli = createCli();
+
+    // Specify an invalid appAttemptId
+    int exitCode = cli.run(new String[] {"-applicationAttemptId", "123"});
+    assertTrue(exitCode == -1);
+    assertTrue(sysErrStream.toString().contains(
+            "Invalid AppAttemptId specified"));
     sysErrStream.reset();
+  }
+
+  @Test (timeout = 5000)
+  public void testWithInvalidContainerId() throws Exception {
+    LogsCLI cli = createCli();
 
     // Specify an invalid containerId
-    exitCode = cli.run(new String[] {"-containerId",
-        "123"});
+    int exitCode = cli.run(new String[] {"-containerId", "123"});
     assertTrue(exitCode == -1);
     assertTrue(sysErrStream.toString().contains(
-        "Invalid ContainerId specified"));
+            "Invalid ContainerId specified"));
+    sysErrStream.reset();
+  }
+
+  @Test (timeout = 5000)
+  public void testWithNonMatchingEntityIds() throws Exception {
+    ApplicationId appId1 = ApplicationId.newInstance(0, 1);
+    ApplicationId appId2 = ApplicationId.newInstance(0, 2);
+    ApplicationAttemptId appAttemptId1 =
+            ApplicationAttemptId.newInstance(appId1, 1);
+    ApplicationAttemptId appAttemptId2 =
+            ApplicationAttemptId.newInstance(appId2, 1);
+    ContainerId containerId0 = ContainerId.newContainerId(appAttemptId1, 0);
+    LogsCLI cli = createCli();
+
+    // Non-matching applicationId and applicationAttemptId
+    int exitCode = cli.run(new String[] {"-applicationId", appId2.toString(),
+        "-applicationAttemptId", appAttemptId1.toString()});
+    assertTrue(exitCode == -1);
+    assertTrue(sysErrStream.toString().contains(
+        "The Application:" + appId2.toString()
+            + " does not have the AppAttempt:" + appAttemptId1.toString()));
     sysErrStream.reset();
 
+    // Non-matching applicationId and containerId
+    exitCode = cli.run(new String[] {"-applicationId", appId2.toString(),
+        "-containerId", containerId0.toString()});
+    assertTrue(exitCode == -1);
+    assertTrue(sysErrStream.toString().contains(
+        "The Application:" + appId2.toString()
+            + " does not have the container:" + containerId0.toString()));
+    sysErrStream.reset();
+
+    // Non-matching applicationAttemptId and containerId
+    exitCode = cli.run(new String[] {"-applicationAttemptId",
+            appAttemptId2.toString(), "-containerId", containerId0.toString()});
+    assertTrue(exitCode == -1);
+    assertTrue(sysErrStream.toString().contains(
+        "The AppAttempt:" + appAttemptId2.toString()
+            + " does not have the container:" + containerId0.toString()));
+    sysErrStream.reset();
+  }
+
+  @Test (timeout = 5000)
+  public void testWithExclusiveArguments() throws Exception {
+    ApplicationId appId1 = ApplicationId.newInstance(0, 1);
+    LogsCLI cli = createCli();
+
     // Specify show_container_log_info and show_application_log_info
     // at the same time
-    exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
+    int exitCode = cli.run(new String[] {"-applicationId", appId1.toString(),
         "-show_container_log_info", "-show_application_log_info"});
     assertTrue(exitCode == -1);
     assertTrue(sysErrStream.toString().contains("Invalid options. "
-        + "Can only accept one of show_application_log_info/"
-        + "show_container_log_info."));
+            + "Can only accept one of show_application_log_info/"
+            + "show_container_log_info."));
     sysErrStream.reset();
 
     // Specify log_files and log_files_pattern
     // at the same time
-    exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
+    exitCode = cli.run(new String[] {"-applicationId", appId1.toString(),
         "-log_files", "*", "-log_files_pattern", ".*"});
     assertTrue(exitCode == -1);
     assertTrue(sysErrStream.toString().contains("Invalid options. "
-        + "Can only accept one of log_files/"
-        + "log_files_pattern."));
+            + "Can only accept one of log_files/"
+            + "log_files_pattern."));
     sysErrStream.reset();
+  }
+
+  @Test (timeout = 5000)
+  public void testWithFileInputForOptionOut() throws Exception {
+    String localDir = "target/SaveLogs";
+    Path localPath = new Path(localDir);
+    FileSystem fs = FileSystem.get(conf);
+    ApplicationId appId1 = ApplicationId.newInstance(0, 1);
+    LogsCLI cli = createCli();
 
     // Specify a file name to the option -out
     try {
@@ -1193,8 +1311,8 @@ public class TestLogsCLI {
       if (!fs.exists(tmpFilePath)) {
         fs.createNewFile(tmpFilePath);
       }
-      exitCode = cli.run(new String[] {"-applicationId",
-          appId.toString(),
+      int exitCode = cli.run(new String[] {"-applicationId",
+          appId1.toString(),
           "-out" , tmpFilePath.toString()});
       assertTrue(exitCode == -1);
       assertTrue(sysErrStream.toString().contains(
@@ -1708,6 +1826,15 @@ public class TestLogsCLI {
     }
   }
 
+  private LogsCLI createCli() throws IOException, YarnException {
+    YarnClient mockYarnClient =
+            createMockYarnClient(YarnApplicationState.FINISHED,
+                    UserGroupInformation.getCurrentUser().getShortUserName());
+    LogsCLI cli = new LogsCLIForTest(mockYarnClient);
+    cli.setConf(conf);
+    return cli;
+  }
+
   private YarnClient createMockYarnClient(YarnApplicationState appState,
       String user) throws YarnException, IOException {
     return createMockYarnClient(appState, user, false, null, null);