|
@@ -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)
|