|
@@ -18,13 +18,7 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
|
|
package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
|
-import java.io.OutputStream;
|
|
|
|
-import java.nio.charset.Charset;
|
|
|
|
-import java.util.ArrayList;
|
|
|
|
import java.util.Collections;
|
|
import java.util.Collections;
|
|
-import java.util.HashSet;
|
|
|
|
-import java.util.List;
|
|
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
|
|
|
|
import javax.servlet.http.HttpServletRequest;
|
|
import javax.servlet.http.HttpServletRequest;
|
|
@@ -35,15 +29,16 @@ import javax.ws.rs.Path;
|
|
import javax.ws.rs.PathParam;
|
|
import javax.ws.rs.PathParam;
|
|
import javax.ws.rs.Produces;
|
|
import javax.ws.rs.Produces;
|
|
import javax.ws.rs.QueryParam;
|
|
import javax.ws.rs.QueryParam;
|
|
-import javax.ws.rs.WebApplicationException;
|
|
|
|
import javax.ws.rs.core.Context;
|
|
import javax.ws.rs.core.Context;
|
|
-import javax.ws.rs.core.GenericEntity;
|
|
|
|
import javax.ws.rs.core.MediaType;
|
|
import javax.ws.rs.core.MediaType;
|
|
import javax.ws.rs.core.Response;
|
|
import javax.ws.rs.core.Response;
|
|
-import javax.ws.rs.core.StreamingOutput;
|
|
|
|
import javax.ws.rs.core.Response.ResponseBuilder;
|
|
import javax.ws.rs.core.Response.ResponseBuilder;
|
|
import javax.ws.rs.core.Response.Status;
|
|
import javax.ws.rs.core.Response.Status;
|
|
-import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
|
|
|
|
+
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
|
+import com.sun.jersey.api.client.ClientHandlerException;
|
|
|
|
+import com.sun.jersey.api.client.UniformInterfaceException;
|
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
|
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -54,10 +49,8 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|
import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
|
|
import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
|
|
-import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
|
|
|
|
-import org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest;
|
|
|
|
import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerFactory;
|
|
import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerFactory;
|
|
-import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
|
|
|
|
|
|
+import org.apache.hadoop.yarn.server.webapp.LogWebServiceUtils;
|
|
import org.apache.hadoop.yarn.server.webapp.WebServices;
|
|
import org.apache.hadoop.yarn.server.webapp.WebServices;
|
|
import org.apache.hadoop.yarn.server.webapp.YarnWebServiceParams;
|
|
import org.apache.hadoop.yarn.server.webapp.YarnWebServiceParams;
|
|
import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
|
|
import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
|
|
@@ -65,21 +58,14 @@ import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo;
|
|
import org.apache.hadoop.yarn.server.webapp.dao.AppInfo;
|
|
import org.apache.hadoop.yarn.server.webapp.dao.AppInfo;
|
|
import org.apache.hadoop.yarn.server.webapp.dao.AppsInfo;
|
|
import org.apache.hadoop.yarn.server.webapp.dao.AppsInfo;
|
|
import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
|
|
import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
|
|
-import org.apache.hadoop.yarn.server.webapp.dao.ContainerLogsInfo;
|
|
|
|
import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
|
|
import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
|
|
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
|
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
|
import org.apache.hadoop.yarn.webapp.BadRequestException;
|
|
import org.apache.hadoop.yarn.webapp.BadRequestException;
|
|
import org.apache.hadoop.yarn.webapp.NotFoundException;
|
|
import org.apache.hadoop.yarn.webapp.NotFoundException;
|
|
-import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
|
|
|
|
-import org.apache.hadoop.yarn.webapp.util.YarnWebServiceUtils;
|
|
|
|
-import org.codehaus.jettison.json.JSONException;
|
|
|
|
-import org.codehaus.jettison.json.JSONObject;
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
|
import com.google.common.base.Joiner;
|
|
import com.google.common.base.Joiner;
|
|
import com.google.inject.Inject;
|
|
import com.google.inject.Inject;
|
|
import com.google.inject.Singleton;
|
|
import com.google.inject.Singleton;
|
|
-import com.sun.jersey.api.client.ClientHandlerException;
|
|
|
|
-import com.sun.jersey.api.client.UniformInterfaceException;
|
|
|
|
|
|
+import org.codehaus.jettison.json.JSONException;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
@@ -271,15 +257,18 @@ public class AHSWebServices extends WebServices {
|
|
appInfo = super.getApp(req, res, appId.toString());
|
|
appInfo = super.getApp(req, res, appId.toString());
|
|
} catch (Exception ex) {
|
|
} catch (Exception ex) {
|
|
// directly find logs from HDFS.
|
|
// directly find logs from HDFS.
|
|
- return getContainerLogMeta(appId, null, null, containerIdStr, false);
|
|
|
|
|
|
+ return LogWebServiceUtils
|
|
|
|
+ .getContainerLogMeta(factory, appId, null, null, containerIdStr,
|
|
|
|
+ false);
|
|
}
|
|
}
|
|
// if the application finishes, directly find logs
|
|
// if the application finishes, directly find logs
|
|
// from HDFS.
|
|
// from HDFS.
|
|
- if (isFinishedState(appInfo.getAppState())) {
|
|
|
|
- return getContainerLogMeta(appId, null, null,
|
|
|
|
- containerIdStr, false);
|
|
|
|
|
|
+ if (LogWebServiceUtils.isFinishedState(appInfo.getAppState())) {
|
|
|
|
+ return LogWebServiceUtils
|
|
|
|
+ .getContainerLogMeta(factory, appId, null, null, containerIdStr,
|
|
|
|
+ false);
|
|
}
|
|
}
|
|
- if (isRunningState(appInfo.getAppState())) {
|
|
|
|
|
|
+ if (LogWebServiceUtils.isRunningState(appInfo.getAppState())) {
|
|
String appOwner = appInfo.getUser();
|
|
String appOwner = appInfo.getUser();
|
|
String nodeHttpAddress = null;
|
|
String nodeHttpAddress = null;
|
|
if (nmId != null && !nmId.isEmpty()) {
|
|
if (nmId != null && !nmId.isEmpty()) {
|
|
@@ -301,8 +290,9 @@ public class AHSWebServices extends WebServices {
|
|
} catch (Exception ex) {
|
|
} catch (Exception ex) {
|
|
// return log meta for the aggregated logs if exists.
|
|
// return log meta for the aggregated logs if exists.
|
|
// It will also return empty log meta for the local logs.
|
|
// It will also return empty log meta for the local logs.
|
|
- return getContainerLogMeta(appId, appOwner, null,
|
|
|
|
- containerIdStr, true);
|
|
|
|
|
|
+ return LogWebServiceUtils
|
|
|
|
+ .getContainerLogMeta(factory, appId, appOwner, null,
|
|
|
|
+ containerIdStr, true);
|
|
}
|
|
}
|
|
nodeHttpAddress = containerInfo.getNodeHttpAddress();
|
|
nodeHttpAddress = containerInfo.getNodeHttpAddress();
|
|
// make sure nodeHttpAddress is not null and not empty. Otherwise,
|
|
// make sure nodeHttpAddress is not null and not empty. Otherwise,
|
|
@@ -314,12 +304,14 @@ public class AHSWebServices extends WebServices {
|
|
// It will also return empty log meta for the local logs.
|
|
// It will also return empty log meta for the local logs.
|
|
// If this is the redirect request from NM, we should not
|
|
// If this is the redirect request from NM, we should not
|
|
// re-direct the request back. Simply output the aggregated log meta.
|
|
// re-direct the request back. Simply output the aggregated log meta.
|
|
- return getContainerLogMeta(appId, appOwner, null,
|
|
|
|
- containerIdStr, true);
|
|
|
|
|
|
+ return LogWebServiceUtils
|
|
|
|
+ .getContainerLogMeta(factory, appId, appOwner, null,
|
|
|
|
+ containerIdStr, true);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
String uri = "/" + containerId.toString() + "/logs";
|
|
String uri = "/" + containerId.toString() + "/logs";
|
|
- String resURI = JOINER.join(getAbsoluteNMWebAddress(nodeHttpAddress),
|
|
|
|
|
|
+ String resURI = JOINER.join(
|
|
|
|
+ LogWebServiceUtils.getAbsoluteNMWebAddress(conf, nodeHttpAddress),
|
|
NM_DOWNLOAD_URI_STR, uri);
|
|
NM_DOWNLOAD_URI_STR, uri);
|
|
String query = req.getQueryString();
|
|
String query = req.getQueryString();
|
|
if (query != null && !query.isEmpty()) {
|
|
if (query != null && !query.isEmpty()) {
|
|
@@ -397,11 +389,11 @@ public class AHSWebServices extends WebServices {
|
|
try {
|
|
try {
|
|
containerId = ContainerId.fromString(containerIdStr);
|
|
containerId = ContainerId.fromString(containerIdStr);
|
|
} catch (IllegalArgumentException ex) {
|
|
} catch (IllegalArgumentException ex) {
|
|
- return createBadResponse(Status.NOT_FOUND,
|
|
|
|
|
|
+ return LogWebServiceUtils.createBadResponse(Status.NOT_FOUND,
|
|
"Invalid ContainerId: " + containerIdStr);
|
|
"Invalid ContainerId: " + containerIdStr);
|
|
}
|
|
}
|
|
|
|
|
|
- final long length = parseLongParam(size);
|
|
|
|
|
|
+ final long length = LogWebServiceUtils.parseLongParam(size);
|
|
|
|
|
|
ApplicationId appId = containerId.getApplicationAttemptId()
|
|
ApplicationId appId = containerId.getApplicationAttemptId()
|
|
.getApplicationId();
|
|
.getApplicationId();
|
|
@@ -410,17 +402,19 @@ public class AHSWebServices extends WebServices {
|
|
appInfo = super.getApp(req, res, appId.toString());
|
|
appInfo = super.getApp(req, res, appId.toString());
|
|
} catch (Exception ex) {
|
|
} catch (Exception ex) {
|
|
// directly find logs from HDFS.
|
|
// directly find logs from HDFS.
|
|
- return sendStreamOutputResponse(appId, null, null, containerIdStr,
|
|
|
|
- filename, format, length, false);
|
|
|
|
|
|
+ return LogWebServiceUtils
|
|
|
|
+ .sendStreamOutputResponse(factory, appId, null, null, containerIdStr,
|
|
|
|
+ filename, format, length, false);
|
|
}
|
|
}
|
|
String appOwner = appInfo.getUser();
|
|
String appOwner = appInfo.getUser();
|
|
- if (isFinishedState(appInfo.getAppState())) {
|
|
|
|
|
|
+ if (LogWebServiceUtils.isFinishedState(appInfo.getAppState())) {
|
|
// directly find logs from HDFS.
|
|
// directly find logs from HDFS.
|
|
- return sendStreamOutputResponse(appId, appOwner, null, containerIdStr,
|
|
|
|
- filename, format, length, false);
|
|
|
|
|
|
+ return LogWebServiceUtils
|
|
|
|
+ .sendStreamOutputResponse(factory, appId, appOwner, null,
|
|
|
|
+ containerIdStr, filename, format, length, false);
|
|
}
|
|
}
|
|
|
|
|
|
- if (isRunningState(appInfo.getAppState())) {
|
|
|
|
|
|
+ if (LogWebServiceUtils.isRunningState(appInfo.getAppState())) {
|
|
String nodeHttpAddress = null;
|
|
String nodeHttpAddress = null;
|
|
if (nmId != null && !nmId.isEmpty()) {
|
|
if (nmId != null && !nmId.isEmpty()) {
|
|
try {
|
|
try {
|
|
@@ -440,8 +434,9 @@ public class AHSWebServices extends WebServices {
|
|
containerId.toString());
|
|
containerId.toString());
|
|
} catch (Exception ex) {
|
|
} catch (Exception ex) {
|
|
// output the aggregated logs
|
|
// output the aggregated logs
|
|
- return sendStreamOutputResponse(appId, appOwner, null,
|
|
|
|
- containerIdStr, filename, format, length, true);
|
|
|
|
|
|
+ return LogWebServiceUtils
|
|
|
|
+ .sendStreamOutputResponse(factory, appId, appOwner, null,
|
|
|
|
+ containerIdStr, filename, format, length, true);
|
|
}
|
|
}
|
|
nodeHttpAddress = containerInfo.getNodeHttpAddress();
|
|
nodeHttpAddress = containerInfo.getNodeHttpAddress();
|
|
// make sure nodeHttpAddress is not null and not empty. Otherwise,
|
|
// make sure nodeHttpAddress is not null and not empty. Otherwise,
|
|
@@ -452,12 +447,14 @@ public class AHSWebServices extends WebServices {
|
|
if (nodeHttpAddress == null || nodeHttpAddress.isEmpty()
|
|
if (nodeHttpAddress == null || nodeHttpAddress.isEmpty()
|
|
|| redirected_from_node) {
|
|
|| redirected_from_node) {
|
|
// output the aggregated logs
|
|
// output the aggregated logs
|
|
- return sendStreamOutputResponse(appId, appOwner, null,
|
|
|
|
- containerIdStr, filename, format, length, true);
|
|
|
|
|
|
+ return LogWebServiceUtils
|
|
|
|
+ .sendStreamOutputResponse(factory, appId, appOwner, null,
|
|
|
|
+ containerIdStr, filename, format, length, true);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
String uri = "/" + containerId.toString() + "/logs/" + filename;
|
|
String uri = "/" + containerId.toString() + "/logs/" + filename;
|
|
- String resURI = JOINER.join(getAbsoluteNMWebAddress(nodeHttpAddress),
|
|
|
|
|
|
+ String resURI = JOINER.join(
|
|
|
|
+ LogWebServiceUtils.getAbsoluteNMWebAddress(conf, nodeHttpAddress),
|
|
NM_DOWNLOAD_URI_STR, uri);
|
|
NM_DOWNLOAD_URI_STR, uri);
|
|
String query = req.getQueryString();
|
|
String query = req.getQueryString();
|
|
if (query != null && !query.isEmpty()) {
|
|
if (query != null && !query.isEmpty()) {
|
|
@@ -468,170 +465,15 @@ public class AHSWebServices extends WebServices {
|
|
response.header("Location", resURI);
|
|
response.header("Location", resURI);
|
|
return response.build();
|
|
return response.build();
|
|
} else {
|
|
} else {
|
|
- return createBadResponse(Status.NOT_FOUND,
|
|
|
|
|
|
+ return LogWebServiceUtils.createBadResponse(Status.NOT_FOUND,
|
|
"The application is not at Running or Finished State.");
|
|
"The application is not at Running or Finished State.");
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private boolean isRunningState(YarnApplicationState appState) {
|
|
|
|
- return appState == YarnApplicationState.RUNNING;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private boolean isFinishedState(YarnApplicationState appState) {
|
|
|
|
- return appState == YarnApplicationState.FINISHED
|
|
|
|
- || appState == YarnApplicationState.FAILED
|
|
|
|
- || appState == YarnApplicationState.KILLED;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private Response createBadResponse(Status status, String errMessage) {
|
|
|
|
- Response response = Response.status(status)
|
|
|
|
- .entity(DOT_JOINER.join(status.toString(), errMessage)).build();
|
|
|
|
- return response;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private Response sendStreamOutputResponse(ApplicationId appId,
|
|
|
|
- String appOwner, String nodeId, String containerIdStr,
|
|
|
|
- String fileName, String format, long bytes,
|
|
|
|
- boolean printEmptyLocalContainerLog) {
|
|
|
|
- String contentType = WebAppUtils.getDefaultLogContentType();
|
|
|
|
- if (format != null && !format.isEmpty()) {
|
|
|
|
- contentType = WebAppUtils.getSupportedLogContentType(format);
|
|
|
|
- if (contentType == null) {
|
|
|
|
- String errorMessage = "The valid values for the parameter : format "
|
|
|
|
- + "are " + WebAppUtils.listSupportedLogContentType();
|
|
|
|
- return Response.status(Status.BAD_REQUEST).entity(errorMessage)
|
|
|
|
- .build();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- StreamingOutput stream = null;
|
|
|
|
- try {
|
|
|
|
- stream = getStreamingOutput(appId, appOwner, nodeId,
|
|
|
|
- containerIdStr, fileName, bytes, printEmptyLocalContainerLog);
|
|
|
|
- } catch (Exception ex) {
|
|
|
|
- return createBadResponse(Status.INTERNAL_SERVER_ERROR,
|
|
|
|
- ex.getMessage());
|
|
|
|
- }
|
|
|
|
- ResponseBuilder response = Response.ok(stream);
|
|
|
|
- response.header("Content-Type", contentType);
|
|
|
|
- // Sending the X-Content-Type-Options response header with the value
|
|
|
|
- // nosniff will prevent Internet Explorer from MIME-sniffing a response
|
|
|
|
- // away from the declared content-type.
|
|
|
|
- response.header("X-Content-Type-Options", "nosniff");
|
|
|
|
- return response.build();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private StreamingOutput getStreamingOutput(final ApplicationId appId,
|
|
|
|
- final String appOwner, final String nodeId, final String containerIdStr,
|
|
|
|
- final String logFile, final long bytes,
|
|
|
|
- final boolean printEmptyLocalContainerLog) throws IOException{
|
|
|
|
- StreamingOutput stream = new StreamingOutput() {
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public void write(OutputStream os) throws IOException,
|
|
|
|
- WebApplicationException {
|
|
|
|
- ContainerLogsRequest request = new ContainerLogsRequest();
|
|
|
|
- request.setAppId(appId);
|
|
|
|
- request.setAppOwner(appOwner);
|
|
|
|
- request.setContainerId(containerIdStr);
|
|
|
|
- request.setBytes(bytes);
|
|
|
|
- request.setNodeId(nodeId);
|
|
|
|
- Set<String> logTypes = new HashSet<>();
|
|
|
|
- logTypes.add(logFile);
|
|
|
|
- request.setLogTypes(logTypes);
|
|
|
|
- boolean findLogs = factory.getFileControllerForRead(appId, appOwner)
|
|
|
|
- .readAggregatedLogs(request, os);
|
|
|
|
- if (!findLogs) {
|
|
|
|
- os.write(("Can not find logs for container:"
|
|
|
|
- + containerIdStr).getBytes(Charset.forName("UTF-8")));
|
|
|
|
- } else {
|
|
|
|
- if (printEmptyLocalContainerLog) {
|
|
|
|
- StringBuilder sb = new StringBuilder();
|
|
|
|
- sb.append(containerIdStr + "\n");
|
|
|
|
- sb.append("LogAggregationType: "
|
|
|
|
- + ContainerLogAggregationType.LOCAL + "\n");
|
|
|
|
- sb.append("LogContents:\n");
|
|
|
|
- sb.append(getNoRedirectWarning() + "\n");
|
|
|
|
- os.write(sb.toString().getBytes(Charset.forName("UTF-8")));
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- };
|
|
|
|
- return stream;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private long parseLongParam(String bytes) {
|
|
|
|
- if (bytes == null || bytes.isEmpty()) {
|
|
|
|
- return Long.MAX_VALUE;
|
|
|
|
- }
|
|
|
|
- return Long.parseLong(bytes);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private Response getContainerLogMeta(ApplicationId appId, String appOwner,
|
|
|
|
- final String nodeId, final String containerIdStr,
|
|
|
|
- boolean emptyLocalContainerLogMeta) {
|
|
|
|
- try {
|
|
|
|
- ContainerLogsRequest request = new ContainerLogsRequest();
|
|
|
|
- request.setAppId(appId);
|
|
|
|
- request.setAppOwner(appOwner);
|
|
|
|
- request.setContainerId(containerIdStr);
|
|
|
|
- request.setNodeId(nodeId);
|
|
|
|
- List<ContainerLogMeta> containerLogMeta = factory
|
|
|
|
- .getFileControllerForRead(appId, appOwner)
|
|
|
|
- .readAggregatedLogsMeta(request);
|
|
|
|
- if (containerLogMeta.isEmpty()) {
|
|
|
|
- throw new NotFoundException(
|
|
|
|
- "Can not get log meta for container: " + containerIdStr);
|
|
|
|
- }
|
|
|
|
- List<ContainerLogsInfo> containersLogsInfo = new ArrayList<>();
|
|
|
|
- for (ContainerLogMeta meta : containerLogMeta) {
|
|
|
|
- ContainerLogsInfo logInfo = new ContainerLogsInfo(meta,
|
|
|
|
- ContainerLogAggregationType.AGGREGATED);
|
|
|
|
- containersLogsInfo.add(logInfo);
|
|
|
|
- }
|
|
|
|
- if (emptyLocalContainerLogMeta) {
|
|
|
|
- ContainerLogMeta emptyMeta = new ContainerLogMeta(
|
|
|
|
- containerIdStr, "N/A");
|
|
|
|
- ContainerLogsInfo empty = new ContainerLogsInfo(emptyMeta,
|
|
|
|
- ContainerLogAggregationType.LOCAL);
|
|
|
|
- containersLogsInfo.add(empty);
|
|
|
|
- }
|
|
|
|
- GenericEntity<List<ContainerLogsInfo>> meta = new GenericEntity<List<
|
|
|
|
- ContainerLogsInfo>>(containersLogsInfo){};
|
|
|
|
- ResponseBuilder response = Response.ok(meta);
|
|
|
|
- // Sending the X-Content-Type-Options response header with the value
|
|
|
|
- // nosniff will prevent Internet Explorer from MIME-sniffing a response
|
|
|
|
- // away from the declared content-type.
|
|
|
|
- response.header("X-Content-Type-Options", "nosniff");
|
|
|
|
- return response.build();
|
|
|
|
- } catch (Exception ex) {
|
|
|
|
- throw new WebApplicationException(ex);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Private
|
|
|
|
- @VisibleForTesting
|
|
|
|
- public static String getNoRedirectWarning() {
|
|
|
|
- return "We do not have NodeManager web address, so we can not "
|
|
|
|
- + "re-direct the request to related NodeManager "
|
|
|
|
- + "for local container logs.";
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private String getAbsoluteNMWebAddress(String nmWebAddress) {
|
|
|
|
- if (nmWebAddress.contains(WebAppUtils.HTTP_PREFIX) ||
|
|
|
|
- nmWebAddress.contains(WebAppUtils.HTTPS_PREFIX)) {
|
|
|
|
- return nmWebAddress;
|
|
|
|
- }
|
|
|
|
- return WebAppUtils.getHttpSchemePrefix(conf) + nmWebAddress;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @VisibleForTesting
|
|
|
|
- @Private
|
|
|
|
|
|
+ @VisibleForTesting @InterfaceAudience.Private
|
|
public String getNMWebAddressFromRM(Configuration configuration,
|
|
public String getNMWebAddressFromRM(Configuration configuration,
|
|
- String nodeId) throws ClientHandlerException,
|
|
|
|
- UniformInterfaceException, JSONException {
|
|
|
|
- JSONObject nodeInfo = YarnWebServiceUtils.getNodeInfoFromRMWebService(
|
|
|
|
- configuration, nodeId).getJSONObject("node");
|
|
|
|
- return nodeInfo.has("nodeHTTPAddress") ?
|
|
|
|
- nodeInfo.getString("nodeHTTPAddress") : null;
|
|
|
|
|
|
+ String nodeId)
|
|
|
|
+ throws ClientHandlerException, UniformInterfaceException, JSONException {
|
|
|
|
+ return LogWebServiceUtils.getNMWebAddressFromRM(configuration, nodeId);
|
|
}
|
|
}
|
|
}
|
|
}
|