|
@@ -29,9 +29,12 @@ import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
|
import java.util.Collections;
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
+import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
+import java.util.Map.Entry;
|
|
|
import java.util.Set;
|
|
|
import java.util.TreeMap;
|
|
|
import java.util.regex.Pattern;
|
|
@@ -107,6 +110,7 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
private static final String CLIENT_RETRY_INTERVAL_OPTION
|
|
|
= "client_retry_interval_ms";
|
|
|
public static final String HELP_CMD = "help";
|
|
|
+ private static final String SIZE_LIMIT_OPTION = "size_limit_mb";
|
|
|
|
|
|
private PrintStream outStream = System.out;
|
|
|
private YarnClient yarnClient = null;
|
|
@@ -115,6 +119,11 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
private static final int DEFAULT_MAX_RETRIES = 30;
|
|
|
private static final long DEFAULT_RETRY_INTERVAL = 1000;
|
|
|
|
|
|
+ private static final long LOG_SIZE_LIMIT_DEFAULT = 10240L;
|
|
|
+
|
|
|
+ private long logSizeLeft = LOG_SIZE_LIMIT_DEFAULT * 1024 * 1024;
|
|
|
+ private long specifedLogLimits = LOG_SIZE_LIMIT_DEFAULT;
|
|
|
+
|
|
|
@Private
|
|
|
@VisibleForTesting
|
|
|
ClientConnectionRetry connectionRetry;
|
|
@@ -158,6 +167,7 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
List<String> amContainersList = new ArrayList<String>();
|
|
|
String localDir = null;
|
|
|
long bytes = Long.MAX_VALUE;
|
|
|
+ boolean ignoreSizeLimit = false;
|
|
|
int maxRetries = DEFAULT_MAX_RETRIES;
|
|
|
long retryInterval = DEFAULT_RETRY_INTERVAL;
|
|
|
try {
|
|
@@ -199,6 +209,14 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
retryInterval = Long.parseLong(commandLine.getOptionValue(
|
|
|
CLIENT_RETRY_INTERVAL_OPTION));
|
|
|
}
|
|
|
+ if (commandLine.hasOption(SIZE_LIMIT_OPTION)) {
|
|
|
+ specifedLogLimits = Long.parseLong(commandLine.getOptionValue(
|
|
|
+ SIZE_LIMIT_OPTION));
|
|
|
+ logSizeLeft = specifedLogLimits * 1024 * 1024;
|
|
|
+ }
|
|
|
+ if (logSizeLeft < 0L) {
|
|
|
+ ignoreSizeLimit = true;
|
|
|
+ }
|
|
|
} catch (ParseException e) {
|
|
|
System.err.println("options parsing failed: " + e.getMessage());
|
|
|
printHelpMessage(printOpts);
|
|
@@ -306,6 +324,7 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
logs.addAll(Arrays.asList(logFilesRegex));
|
|
|
}
|
|
|
|
|
|
+
|
|
|
ContainerLogsRequest request = new ContainerLogsRequest(appId,
|
|
|
isApplicationFinished(appState), appOwner, nodeAddress, null,
|
|
|
containerIdStr, localDir, logs, bytes, null);
|
|
@@ -324,15 +343,17 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
// To get am logs
|
|
|
if (getAMContainerLogs) {
|
|
|
return fetchAMContainerLogs(request, amContainersList,
|
|
|
- logCliHelper, useRegex);
|
|
|
+ logCliHelper, useRegex, ignoreSizeLimit);
|
|
|
}
|
|
|
|
|
|
int resultCode = 0;
|
|
|
if (containerIdStr != null) {
|
|
|
- return fetchContainerLogs(request, logCliHelper, useRegex);
|
|
|
+ return fetchContainerLogs(request, logCliHelper, useRegex,
|
|
|
+ ignoreSizeLimit);
|
|
|
} else {
|
|
|
if (nodeAddress == null) {
|
|
|
- resultCode = fetchApplicationLogs(request, logCliHelper, useRegex);
|
|
|
+ resultCode = fetchApplicationLogs(request, logCliHelper, useRegex,
|
|
|
+ ignoreSizeLimit);
|
|
|
} else {
|
|
|
System.err.println("Should at least provide ContainerId!");
|
|
|
printHelpMessage(printOpts);
|
|
@@ -524,35 +545,16 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
@VisibleForTesting
|
|
|
public int printContainerLogsFromRunningApplication(Configuration conf,
|
|
|
ContainerLogsRequest request, LogCLIHelpers logCliHelper,
|
|
|
- boolean useRegex) throws IOException {
|
|
|
+ boolean useRegex, boolean ignoreSizeLimit) throws IOException {
|
|
|
String containerIdStr = request.getContainerId().toString();
|
|
|
String localDir = request.getOutputLocalDir();
|
|
|
- String nodeHttpAddress = request.getNodeHttpAddress();
|
|
|
- if (nodeHttpAddress == null || nodeHttpAddress.isEmpty()) {
|
|
|
- System.err.println("Can not get the logs for the container: "
|
|
|
- + containerIdStr);
|
|
|
- System.err.println("The node http address is required to get container "
|
|
|
- + "logs for the Running application.");
|
|
|
- return -1;
|
|
|
- }
|
|
|
String nodeId = request.getNodeId();
|
|
|
PrintStream out = LogToolUtils.createPrintStream(localDir, nodeId,
|
|
|
containerIdStr);
|
|
|
try {
|
|
|
- Set<String> matchedFiles = getMatchedContainerLogFiles(request,
|
|
|
- useRegex);
|
|
|
- if (matchedFiles.isEmpty()) {
|
|
|
- System.err.println("Can not find any log file matching the pattern: "
|
|
|
- + request.getLogTypes() + " for the container: " + containerIdStr
|
|
|
- + " within the application: " + request.getAppId());
|
|
|
- return -1;
|
|
|
- }
|
|
|
- ContainerLogsRequest newOptions = new ContainerLogsRequest(request);
|
|
|
- newOptions.setLogTypes(matchedFiles);
|
|
|
-
|
|
|
boolean foundAnyLogs = false;
|
|
|
byte[] buffer = new byte[65536];
|
|
|
- for (String logFile : newOptions.getLogTypes()) {
|
|
|
+ for (String logFile : request.getLogTypes()) {
|
|
|
InputStream is = null;
|
|
|
try {
|
|
|
ClientResponse response = getResponeFromNMWebService(conf,
|
|
@@ -595,50 +597,6 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private int printContainerLogsForFinishedApplication(
|
|
|
- ContainerLogsRequest request, LogCLIHelpers logCliHelper,
|
|
|
- boolean useRegex) throws IOException {
|
|
|
- ContainerLogsRequest newOptions = getMatchedLogOptions(
|
|
|
- request, logCliHelper, useRegex);
|
|
|
- if (newOptions == null) {
|
|
|
- System.err.println("Can not find any log file matching the pattern: "
|
|
|
- + request.getLogTypes() + " for the container: "
|
|
|
- + request.getContainerId() + " within the application: "
|
|
|
- + request.getAppId());
|
|
|
- return -1;
|
|
|
- }
|
|
|
- return logCliHelper.dumpAContainerLogsForLogType(newOptions);
|
|
|
- }
|
|
|
-
|
|
|
- private int printContainerLogsForFinishedApplicationWithoutNodeId(
|
|
|
- ContainerLogsRequest request, LogCLIHelpers logCliHelper,
|
|
|
- boolean useRegex) throws IOException {
|
|
|
- ContainerLogsRequest newOptions = getMatchedLogOptions(
|
|
|
- request, logCliHelper, useRegex);
|
|
|
- if (newOptions == null) {
|
|
|
- System.err.println("Can not find any log file matching the pattern: "
|
|
|
- + request.getLogTypes() + " for the container: "
|
|
|
- + request.getContainerId() + " within the application: "
|
|
|
- + request.getAppId());
|
|
|
- return -1;
|
|
|
- }
|
|
|
- return logCliHelper.dumpAContainerLogsForLogTypeWithoutNodeId(
|
|
|
- newOptions);
|
|
|
- }
|
|
|
-
|
|
|
- private int printAggregatedContainerLogs(ContainerLogsRequest request,
|
|
|
- LogCLIHelpers logCliHelper, boolean useRegex) throws IOException {
|
|
|
- return printContainerLogsForFinishedApplication(request,
|
|
|
- logCliHelper, useRegex);
|
|
|
- }
|
|
|
-
|
|
|
- private int printAggregatedContainerLogsWithoutNodeId(
|
|
|
- ContainerLogsRequest request, LogCLIHelpers logCliHelper,
|
|
|
- boolean useRegex) throws IOException {
|
|
|
- return printContainerLogsForFinishedApplicationWithoutNodeId(request,
|
|
|
- logCliHelper, useRegex);
|
|
|
- }
|
|
|
-
|
|
|
@Private
|
|
|
@VisibleForTesting
|
|
|
public ContainerReport getContainerReport(String containerIdStr)
|
|
@@ -655,7 +613,8 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
|
|
|
private int printAMContainerLogs(Configuration conf,
|
|
|
ContainerLogsRequest request, List<String> amContainers,
|
|
|
- LogCLIHelpers logCliHelper, boolean useRegex) throws Exception {
|
|
|
+ LogCLIHelpers logCliHelper, boolean useRegex, boolean ignoreSizeLimit)
|
|
|
+ throws Exception {
|
|
|
List<JSONObject> amContainersList = null;
|
|
|
List<ContainerLogsRequest> requests =
|
|
|
new ArrayList<ContainerLogsRequest>();
|
|
@@ -717,10 +676,9 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
return -1;
|
|
|
}
|
|
|
|
|
|
+ List<ContainerLogsRequest> candidates = new ArrayList<>();
|
|
|
if (amContainers.contains("ALL")) {
|
|
|
- for (ContainerLogsRequest amRequest : requests) {
|
|
|
- outputAMContainerLogs(amRequest, conf, logCliHelper, useRegex);
|
|
|
- }
|
|
|
+ candidates.addAll(requests);
|
|
|
outStream.println();
|
|
|
outStream.println("Specified ALL for -am option. "
|
|
|
+ "Printed logs for all am containers.");
|
|
@@ -728,12 +686,10 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
for (String amContainer : amContainers) {
|
|
|
int amContainerId = Integer.parseInt(amContainer.trim());
|
|
|
if (amContainerId == -1) {
|
|
|
- outputAMContainerLogs(requests.get(requests.size() - 1), conf,
|
|
|
- logCliHelper, useRegex);
|
|
|
+ candidates.add(requests.get(requests.size() - 1));
|
|
|
} else {
|
|
|
if (amContainerId <= requests.size()) {
|
|
|
- outputAMContainerLogs(requests.get(amContainerId - 1), conf,
|
|
|
- logCliHelper, useRegex);
|
|
|
+ candidates.add(requests.get(amContainerId - 1));
|
|
|
} else {
|
|
|
System.err.println(String.format("ERROR: Specified AM containerId"
|
|
|
+ " (%s) exceeds the number of AM containers (%s).",
|
|
@@ -743,12 +699,25 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+ Map<String, ContainerLogsRequest> newOptions = new HashMap<>();
|
|
|
+ if (request.isAppFinished()) {
|
|
|
+ newOptions = getMatchedLogTypesForFinishedApp(candidates,
|
|
|
+ logCliHelper, useRegex, ignoreSizeLimit);
|
|
|
+ } else {
|
|
|
+ newOptions = getMatchedLogTypesForRunningApp(candidates, useRegex,
|
|
|
+ ignoreSizeLimit);
|
|
|
+ }
|
|
|
+ for (Entry<String, ContainerLogsRequest> amRequest
|
|
|
+ : newOptions.entrySet()) {
|
|
|
+ outputAMContainerLogs(amRequest.getValue(), conf, logCliHelper,
|
|
|
+ useRegex, ignoreSizeLimit);
|
|
|
+ }
|
|
|
return 0;
|
|
|
}
|
|
|
|
|
|
private void outputAMContainerLogs(ContainerLogsRequest request,
|
|
|
- Configuration conf, LogCLIHelpers logCliHelper, boolean useRegex)
|
|
|
- throws Exception {
|
|
|
+ Configuration conf, LogCLIHelpers logCliHelper, boolean useRegex,
|
|
|
+ boolean ignoreSizeLimit) throws Exception {
|
|
|
String nodeHttpAddress = request.getNodeHttpAddress();
|
|
|
String containerId = request.getContainerId();
|
|
|
String nodeId = request.getNodeId();
|
|
@@ -756,11 +725,10 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
if (request.isAppFinished()) {
|
|
|
if (containerId != null && !containerId.isEmpty()) {
|
|
|
if (nodeId != null && !nodeId.isEmpty()) {
|
|
|
- printContainerLogsForFinishedApplication(request,
|
|
|
- logCliHelper, useRegex);
|
|
|
+ logCliHelper.dumpAContainerLogsForLogType(request);
|
|
|
} else {
|
|
|
- printContainerLogsForFinishedApplicationWithoutNodeId(
|
|
|
- request, logCliHelper, useRegex);
|
|
|
+ logCliHelper.dumpAContainerLogsForLogTypeWithoutNodeId(
|
|
|
+ request);
|
|
|
}
|
|
|
}
|
|
|
} else {
|
|
@@ -770,7 +738,7 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
.getContainerState();
|
|
|
request.setContainerState(containerState);
|
|
|
printContainerLogsFromRunningApplication(conf,
|
|
|
- request, logCliHelper, useRegex);
|
|
|
+ request, logCliHelper, useRegex, ignoreSizeLimit);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -898,6 +866,13 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
opts.addOption(CLIENT_RETRY_INTERVAL_OPTION, true,
|
|
|
"Work with --client_max_retries to create a retry client. "
|
|
|
+ "The default value is 1000.");
|
|
|
+ opts.addOption(SIZE_LIMIT_OPTION, true, "Use this option to limit "
|
|
|
+ + "the size of the total logs which could be fetched. "
|
|
|
+ + "By default, we only allow to fetch at most "
|
|
|
+ + LOG_SIZE_LIMIT_DEFAULT + " MB logs. If the total log size is "
|
|
|
+ + "larger than the specified number, the CLI would fail. "
|
|
|
+ + "The user could specify -1 to ignore the size limit "
|
|
|
+ + "and fetch all logs.");
|
|
|
opts.getOption(APPLICATION_ID_OPTION).setArgName("Application ID");
|
|
|
opts.getOption(CONTAINER_ID_OPTION).setArgName("Container ID");
|
|
|
opts.getOption(NODE_ADDRESS_OPTION).setArgName("Node Address");
|
|
@@ -908,6 +883,7 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
opts.getOption(CLIENT_MAX_RETRY_OPTION).setArgName("Max Retries");
|
|
|
opts.getOption(CLIENT_RETRY_INTERVAL_OPTION)
|
|
|
.setArgName("Retry Interval");
|
|
|
+ opts.getOption(SIZE_LIMIT_OPTION).setArgName("Size Limit");
|
|
|
return opts;
|
|
|
}
|
|
|
|
|
@@ -933,6 +909,7 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
PER_CONTAINER_LOG_FILES_REGEX_OPTION));
|
|
|
printOpts.addOption(commandOpts.getOption(CLIENT_MAX_RETRY_OPTION));
|
|
|
printOpts.addOption(commandOpts.getOption(CLIENT_RETRY_INTERVAL_OPTION));
|
|
|
+ printOpts.addOption(commandOpts.getOption(SIZE_LIMIT_OPTION));
|
|
|
return printOpts;
|
|
|
}
|
|
|
|
|
@@ -969,15 +946,15 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
|
|
|
private int fetchAMContainerLogs(ContainerLogsRequest request,
|
|
|
List<String> amContainersList, LogCLIHelpers logCliHelper,
|
|
|
- boolean useRegex) throws Exception {
|
|
|
+ boolean useRegex, boolean ignoreSizeLimit) throws Exception {
|
|
|
return printAMContainerLogs(getConf(), request, amContainersList,
|
|
|
- logCliHelper, useRegex);
|
|
|
+ logCliHelper, useRegex, ignoreSizeLimit);
|
|
|
}
|
|
|
|
|
|
private int fetchContainerLogs(ContainerLogsRequest request,
|
|
|
- LogCLIHelpers logCliHelper, boolean useRegex) throws IOException,
|
|
|
- ClientHandlerException, UniformInterfaceException, JSONException {
|
|
|
- int resultCode = 0;
|
|
|
+ LogCLIHelpers logCliHelper, boolean useRegex, boolean ignoreSizeLimit)
|
|
|
+ throws IOException, ClientHandlerException, UniformInterfaceException,
|
|
|
+ JSONException {
|
|
|
String appIdStr = request.getAppId().toString();
|
|
|
String containerIdStr = request.getContainerId();
|
|
|
String nodeAddress = request.getNodeId();
|
|
@@ -988,12 +965,20 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
if (isAppFinished) {
|
|
|
// if user specified "ALL" as the logFiles param, pass empty list
|
|
|
// to logCliHelper so that it fetches all the logs
|
|
|
+ ContainerLogsRequest newOptions = getMatchedLogOptions(
|
|
|
+ request, logCliHelper, useRegex, ignoreSizeLimit);
|
|
|
+ if (newOptions == null) {
|
|
|
+ System.err.println("Can not find any log file matching the pattern: "
|
|
|
+ + request.getLogTypes() + " for the container: "
|
|
|
+ + request.getContainerId() + " within the application: "
|
|
|
+ + request.getAppId());
|
|
|
+ return -1;
|
|
|
+ }
|
|
|
if (nodeAddress != null && !nodeAddress.isEmpty()) {
|
|
|
- return printContainerLogsForFinishedApplication(
|
|
|
- request, logCliHelper, useRegex);
|
|
|
+ return logCliHelper.dumpAContainerLogsForLogType(newOptions);
|
|
|
} else {
|
|
|
- return printContainerLogsForFinishedApplicationWithoutNodeId(
|
|
|
- request, logCliHelper, useRegex);
|
|
|
+ return logCliHelper.dumpAContainerLogsForLogTypeWithoutNodeId(
|
|
|
+ newOptions);
|
|
|
}
|
|
|
}
|
|
|
String nodeHttpAddress = null;
|
|
@@ -1019,13 +1004,20 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
} else {
|
|
|
// for the case, we have already uploaded partial logs in HDFS
|
|
|
int result = -1;
|
|
|
- if (nodeAddress != null && !nodeAddress.isEmpty()) {
|
|
|
- result = printAggregatedContainerLogs(request,
|
|
|
- logCliHelper, useRegex);
|
|
|
+ ContainerLogsRequest newOptions = getMatchedLogOptions(
|
|
|
+ request, logCliHelper, useRegex, ignoreSizeLimit);
|
|
|
+ if (newOptions == null) {
|
|
|
+ System.err.println("Can not find any log file matching the pattern: "
|
|
|
+ + request.getLogTypes() + " for the container: "
|
|
|
+ + request.getContainerId() + " within the application: "
|
|
|
+ + request.getAppId());
|
|
|
} else {
|
|
|
- result = printAggregatedContainerLogsWithoutNodeId(request,
|
|
|
- logCliHelper,
|
|
|
- useRegex);
|
|
|
+ if (nodeAddress != null && !nodeAddress.isEmpty()) {
|
|
|
+ result = logCliHelper.dumpAContainerLogsForLogType(newOptions);
|
|
|
+ } else {
|
|
|
+ result = logCliHelper.dumpAContainerLogsForLogTypeWithoutNodeId(
|
|
|
+ newOptions);
|
|
|
+ }
|
|
|
}
|
|
|
if (result == -1) {
|
|
|
System.err.println(
|
|
@@ -1043,14 +1035,18 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
// If the application is not in the final state,
|
|
|
// we will provide the NodeHttpAddress and get the container logs
|
|
|
// by calling NodeManager webservice.
|
|
|
- resultCode = printContainerLogsFromRunningApplication(getConf(), request,
|
|
|
- logCliHelper, useRegex);
|
|
|
- return resultCode;
|
|
|
+ ContainerLogsRequest newRequest = getMatchedOptionForRunningApp(
|
|
|
+ request, useRegex, ignoreSizeLimit);
|
|
|
+ if (newRequest == null) {
|
|
|
+ return -1;
|
|
|
+ }
|
|
|
+ return printContainerLogsFromRunningApplication(getConf(), request,
|
|
|
+ logCliHelper, useRegex, ignoreSizeLimit);
|
|
|
}
|
|
|
|
|
|
private int fetchApplicationLogs(ContainerLogsRequest options,
|
|
|
- LogCLIHelpers logCliHelper, boolean useRegex) throws IOException,
|
|
|
- YarnException {
|
|
|
+ LogCLIHelpers logCliHelper, boolean useRegex, boolean ignoreSizeLimit)
|
|
|
+ throws IOException, YarnException {
|
|
|
// If the application has finished, we would fetch the logs
|
|
|
// from HDFS.
|
|
|
// If the application is still running, we would get the full
|
|
@@ -1059,7 +1055,7 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
int resultCode = -1;
|
|
|
if (options.isAppFinished()) {
|
|
|
ContainerLogsRequest newOptions = getMatchedLogOptions(
|
|
|
- options, logCliHelper, useRegex);
|
|
|
+ options, logCliHelper, useRegex, ignoreSizeLimit);
|
|
|
if (newOptions == null) {
|
|
|
System.err.println("Can not find any log file matching the pattern: "
|
|
|
+ options.getLogTypes() + " for the application: "
|
|
@@ -1071,9 +1067,17 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
} else {
|
|
|
List<ContainerLogsRequest> containerLogRequests =
|
|
|
getContainersLogRequestForRunningApplication(options);
|
|
|
- for (ContainerLogsRequest container : containerLogRequests) {
|
|
|
+
|
|
|
+ // get all matched container log types and check the total log size.
|
|
|
+ Map<String, ContainerLogsRequest> matchedLogTypes =
|
|
|
+ getMatchedLogTypesForRunningApp(containerLogRequests,
|
|
|
+ useRegex, ignoreSizeLimit);
|
|
|
+
|
|
|
+ for (Entry<String, ContainerLogsRequest> container
|
|
|
+ : matchedLogTypes.entrySet()) {
|
|
|
int result = printContainerLogsFromRunningApplication(getConf(),
|
|
|
- container, logCliHelper, useRegex);
|
|
|
+ container.getValue(), logCliHelper,
|
|
|
+ useRegex, ignoreSizeLimit);
|
|
|
if (result == 0) {
|
|
|
resultCode = 0;
|
|
|
}
|
|
@@ -1103,37 +1107,54 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
|
|
|
private ContainerLogsRequest getMatchedLogOptions(
|
|
|
ContainerLogsRequest request, LogCLIHelpers logCliHelper,
|
|
|
- boolean useRegex) throws IOException {
|
|
|
+ boolean useRegex, boolean ignoreSizeLimit) throws IOException {
|
|
|
ContainerLogsRequest newOptions = new ContainerLogsRequest(request);
|
|
|
- if (request.getLogTypes() != null && !request.getLogTypes().isEmpty()) {
|
|
|
- Set<String> matchedFiles = new HashSet<String>();
|
|
|
- if (!request.getLogTypes().contains("ALL")) {
|
|
|
- Set<String> files = logCliHelper.listContainerLogs(request);
|
|
|
- matchedFiles = getMatchedLogFiles(request, files, useRegex);
|
|
|
- if (matchedFiles.isEmpty()) {
|
|
|
- return null;
|
|
|
- }
|
|
|
- }
|
|
|
+ Set<ContainerLogFileInfo> files = logCliHelper.listContainerLogs(
|
|
|
+ request);
|
|
|
+ Set<String> matchedFiles = getMatchedLogFiles(request, files,
|
|
|
+ useRegex, ignoreSizeLimit);
|
|
|
+ if (matchedFiles.isEmpty()) {
|
|
|
+ return null;
|
|
|
+ } else {
|
|
|
newOptions.setLogTypes(matchedFiles);
|
|
|
+ return newOptions;
|
|
|
}
|
|
|
- return newOptions;
|
|
|
}
|
|
|
|
|
|
private Set<String> getMatchedLogFiles(ContainerLogsRequest options,
|
|
|
- Collection<String> candidate, boolean useRegex) throws IOException {
|
|
|
+ Collection<ContainerLogFileInfo> candidate, boolean useRegex,
|
|
|
+ boolean ignoreSizeLimit) throws IOException {
|
|
|
Set<String> matchedFiles = new HashSet<String>();
|
|
|
Set<String> filePattern = options.getLogTypes();
|
|
|
- if (options.getLogTypes().contains("ALL")) {
|
|
|
- return new HashSet<String>(candidate);
|
|
|
- }
|
|
|
- for (String file : candidate) {
|
|
|
- if (useRegex) {
|
|
|
- if (isFileMatching(file, filePattern)) {
|
|
|
- matchedFiles.add(file);
|
|
|
+ long size = options.getBytes();
|
|
|
+ boolean getAll = options.getLogTypes().contains("ALL");
|
|
|
+ Iterator<ContainerLogFileInfo> iterator = candidate.iterator();
|
|
|
+ while(iterator.hasNext()) {
|
|
|
+ boolean matchedFile = false;
|
|
|
+ ContainerLogFileInfo logInfo = iterator.next();
|
|
|
+ if (getAll) {
|
|
|
+ matchedFile = true;
|
|
|
+ } else if (useRegex) {
|
|
|
+ if (isFileMatching(logInfo.getFileName(), filePattern)) {
|
|
|
+ matchedFile = true;
|
|
|
}
|
|
|
} else {
|
|
|
- if (filePattern.contains(file)) {
|
|
|
- matchedFiles.add(file);
|
|
|
+ if (filePattern.contains(logInfo.getFileName())) {
|
|
|
+ matchedFile = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (matchedFile) {
|
|
|
+ matchedFiles.add(logInfo.getFileName());
|
|
|
+ if (!ignoreSizeLimit) {
|
|
|
+ decrLogSizeLimit(Math.min(
|
|
|
+ Long.parseLong(logInfo.getFileSize()), size));
|
|
|
+ if (getLogSizeLimitLeft() < 0) {
|
|
|
+ throw new RuntimeException("The total log size is too large."
|
|
|
+ + "The log size limit is " + specifedLogLimits + "MB. "
|
|
|
+ + "Please specify a proper value --size option or if you "
|
|
|
+ + "really want to fetch all, please "
|
|
|
+ + "specify -1 for --size_limit_mb option.");
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -1296,18 +1317,19 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
|
|
|
@VisibleForTesting
|
|
|
public Set<String> getMatchedContainerLogFiles(ContainerLogsRequest request,
|
|
|
- boolean useRegex) throws IOException {
|
|
|
+ boolean useRegex, boolean ignoreSizeLimit) throws IOException {
|
|
|
// fetch all the log files for the container
|
|
|
// filter the log files based on the given -log_files pattern
|
|
|
List<Pair<ContainerLogFileInfo, String>> allLogFileInfos=
|
|
|
getContainerLogFiles(getConf(), request.getContainerId(),
|
|
|
request.getNodeHttpAddress());
|
|
|
- List<String> fileNames = new ArrayList<String>();
|
|
|
+ List<ContainerLogFileInfo> fileNames = new ArrayList<
|
|
|
+ ContainerLogFileInfo>();
|
|
|
for (Pair<ContainerLogFileInfo, String> fileInfo : allLogFileInfos) {
|
|
|
- fileNames.add(fileInfo.getKey().getFileName());
|
|
|
+ fileNames.add(fileInfo.getKey());
|
|
|
}
|
|
|
return getMatchedLogFiles(request, fileNames,
|
|
|
- useRegex);
|
|
|
+ useRegex, ignoreSizeLimit);
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@@ -1451,4 +1473,86 @@ public class LogsCLI extends Configured implements Tool {
|
|
|
// The method to indicate if we should retry given the incoming exception
|
|
|
public abstract boolean shouldRetryOn(Exception e);
|
|
|
}
|
|
|
+
|
|
|
+ private long getLogSizeLimitLeft() {
|
|
|
+ return this.logSizeLeft;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void decrLogSizeLimit(long used) {
|
|
|
+ this.logSizeLeft -= used;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Private
|
|
|
+ @VisibleForTesting
|
|
|
+ public ContainerLogsRequest getMatchedOptionForRunningApp(
|
|
|
+ ContainerLogsRequest container, boolean useRegex,
|
|
|
+ boolean ignoreSizeLimit) throws IOException {
|
|
|
+ String containerIdStr = container.getContainerId().toString();
|
|
|
+ String nodeHttpAddress = container.getNodeHttpAddress();
|
|
|
+ if (nodeHttpAddress == null || nodeHttpAddress.isEmpty()) {
|
|
|
+ System.err.println("Can not get the logs for the container: "
|
|
|
+ + containerIdStr);
|
|
|
+ System.err.println("The node http address is required to get container "
|
|
|
+ + "logs for the Running application.");
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ Set<String> matchedFiles = getMatchedContainerLogFiles(container,
|
|
|
+ useRegex, ignoreSizeLimit);
|
|
|
+ if (matchedFiles.isEmpty()) {
|
|
|
+ System.err.println("Can not find any log file matching the pattern: "
|
|
|
+ + container.getLogTypes() + " for the container: " + containerIdStr
|
|
|
+ + " within the application: " + container.getAppId());
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ container.setLogTypes(matchedFiles);
|
|
|
+ return container;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Private
|
|
|
+ @VisibleForTesting
|
|
|
+ public Map<String, ContainerLogsRequest> getMatchedLogTypesForRunningApp(
|
|
|
+ List<ContainerLogsRequest> containerLogRequests, boolean useRegex,
|
|
|
+ boolean ignoreSizeLimit) {
|
|
|
+ Map<String, ContainerLogsRequest> containerMatchedLog = new HashMap<>();
|
|
|
+ for (ContainerLogsRequest container : containerLogRequests) {
|
|
|
+ try {
|
|
|
+ ContainerLogsRequest request = getMatchedOptionForRunningApp(
|
|
|
+ container, useRegex, ignoreSizeLimit);
|
|
|
+ if (request == null) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ containerMatchedLog.put(container.getContainerId(), request);
|
|
|
+ } catch(IOException ex) {
|
|
|
+ System.err.println(ex);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return containerMatchedLog;
|
|
|
+ }
|
|
|
+
|
|
|
+ private Map<String, ContainerLogsRequest> getMatchedLogTypesForFinishedApp(
|
|
|
+ List<ContainerLogsRequest> containerLogRequests,
|
|
|
+ LogCLIHelpers logCliHelper, boolean useRegex,
|
|
|
+ boolean ignoreSizeLimit) {
|
|
|
+ Map<String, ContainerLogsRequest> containerMatchedLog = new HashMap<>();
|
|
|
+ for (ContainerLogsRequest container : containerLogRequests) {
|
|
|
+ try {
|
|
|
+ ContainerLogsRequest request = getMatchedLogOptions(container,
|
|
|
+ logCliHelper, useRegex, ignoreSizeLimit);
|
|
|
+ if (request == null) {
|
|
|
+ System.err.println("Can not find any log file matching the pattern: "
|
|
|
+ + container.getLogTypes() + " for the container: "
|
|
|
+ + container.getContainerId() + " within the application: "
|
|
|
+ + container.getAppId());
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ containerMatchedLog.put(container.getContainerId(), request);
|
|
|
+ } catch (IOException ex) {
|
|
|
+ System.err.println(ex);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return containerMatchedLog;
|
|
|
+ }
|
|
|
}
|