|
@@ -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;
|
|
|
}
|
|
|
}
|